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

Scheduler is unable to find serialized DAG in the serialized_dag table #13504

Closed
arch-DJ opened this issue Jan 6, 2021 · 13 comments · Fixed by #13893
Closed

Scheduler is unable to find serialized DAG in the serialized_dag table #13504

arch-DJ opened this issue Jan 6, 2021 · 13 comments · Fixed by #13893
Assignees
Labels
affected_version:2.0 Issues Reported for 2.0 area:serialization kind:bug This is a clearly a bug priority:medium Bug that should be fixed before next release but would not block a release
Milestone

Comments

@arch-DJ
Copy link

arch-DJ commented Jan 6, 2021

Apache Airflow version: 2.0.0

Kubernetes version (if you are using kubernetes) (use kubectl version): Not relevant

Environment:

  • Cloud provider or hardware configuration:

  • OS (e.g. from /etc/os-release): CentOS Linux 7 (Core)

  • Kernel (e.g. uname -a): Linux us01odcres-jamuaar-0003 3.10.0-957.5.1.el7.x86_64 Improving the search functionality in the graph view #1 SMP Fri Feb 1 14:54:57 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux

  • Install tools: PostgreSQL 12.2

  • Others:

What happened:

I have 2 dag files say, dag1.py and dag2.py.
dag1.py creates a static DAG i.e. once it's parsed it will create 1 specific DAG.
dag2.py creates dynamic DAGs based on json files kept in an external location.

The static DAG (generated from dag1.py) has a task in the later stage which generates json files and they get picked up by dag2.py which creates dynamic DAGs.

The dynamic DAGs which get created are unpaused by default and get scheduled once.
This whole process used to work fine with airflow 1.x where DAG serialization was not mandatory and was turned off by default.

But with Airflow 2.0 I am getting the following exception occasionally when the dynamically generated DAGs try to get scheduled by the scheduler.

[2021-01-06 10:09:38,742] {scheduler_job.py:1293} ERROR - Exception when executing SchedulerJob._run_scheduler_loop
Traceback (most recent call last):
  File "/global/packages/python/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 1275, in _execute
    self._run_scheduler_loop()
  File "/global/packages/python/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 1377, in _run_scheduler_loop
    num_queued_tis = self._do_scheduling(session)
  File "/global/packages/python/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 1474, in _do_scheduling
    self._create_dag_runs(query.all(), session)
  File "/global/packages/python/lib/python3.7/site-packages/airflow/jobs/scheduler_job.py", line 1557, in _create_dag_runs
    dag = self.dagbag.get_dag(dag_model.dag_id, session=session)
  File "/global/packages/python/lib/python3.7/site-packages/airflow/utils/session.py", line 62, in wrapper
    return func(*args, **kwargs)
  File "/global/packages/python/lib/python3.7/site-packages/airflow/models/dagbag.py", line 171, in get_dag
    self._add_dag_from_db(dag_id=dag_id, session=session)
  File "/global/packages/python/lib/python3.7/site-packages/airflow/models/dagbag.py", line 227, in _add_dag_from_db
    raise SerializedDagNotFound(f"DAG '{dag_id}' not found in serialized_dag table")
airflow.exceptions.SerializedDagNotFound: DAG 'dynamic_dag_1' not found in serialized_dag table

When I checked the serialized_dag table manually, I am able to see the DAG entry there.
I found the last_updated column value to be 2021-01-06 10:09:38.757076+05:30
Whereas the exception got logged at [2021-01-06 10:09:38,742] which is little before the last_updated time.

I think this means that the Scheduler tried to look for the DAG entry in the serialized_dag table before DagFileProcessor created the entry.

Is this right or something else can be going on here?

What you expected to happen:

Scheduler should start looking for the DAG entry in the serialized_dag table only after DagFileProcessor has added it.
Here it seems that DagFileProcessor added the DAG entry in the dag table, scheduler immediately fetched this dag_id from it and tried to find the same in serialized_dag table even before DagFileProcessor could add that.

How to reproduce it:
It occurs occasionally and there is no well defined way to reproduce it.

Anything else we need to know:

@arch-DJ arch-DJ added the kind:bug This is a clearly a bug label Jan 6, 2021
@boring-cyborg
Copy link

boring-cyborg bot commented Jan 6, 2021

Thanks for opening your first issue here! Be sure to follow the issue template!

@potiuk potiuk added this to the Airflow 2.0.1 milestone Jan 7, 2021
@potiuk potiuk added the priority:medium Bug that should be fixed before next release but would not block a release label Jan 7, 2021
@doowhtron
Copy link

I have a similar problem. After the "airflow.exceptions.SerializedDagNotFound: DAG 'XXX' not found in serialized_dag table" is logged, the scheduler dies.

@adamtay82
Copy link

Can confirm similar for us.

@doowhtron
Copy link

I temporarily fix this by catching SerializedDagNotFound Exception in scheduler_job.py

from airflow.exceptions import SerializedDagNotFound
for dag_run in dag_runs:
    try:
        self._schedule_dag_run(dag_run, active_runs_by_dag_id.get(dag_run.dag_id, set()), session)
    except SerializedDagNotFound as e:
        self.log.exception(e)

@iameugenejo
Copy link

iameugenejo commented Jan 18, 2021

Experiencing the error.

Python version: 3.8.0
Airflow version: 2.0.0
Node: {REDACTED}
-------------------------------------------------------------------------------
Traceback (most recent call last):
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/flask/app.py", line 2447, in wsgi_app
    response = self.full_dispatch_request()
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/flask/app.py", line 1952, in full_dispatch_request
    rv = self.handle_user_exception(e)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/flask/app.py", line 1821, in handle_user_exception
    reraise(exc_type, exc_value, tb)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/flask/_compat.py", line 39, in reraise
    raise value
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/flask/app.py", line 1950, in full_dispatch_request
    rv = self.dispatch_request()
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/flask/app.py", line 1936, in dispatch_request
    return self.view_functions[rule.endpoint](**req.view_args)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/www/auth.py", line 34, in decorated
    return func(*args, **kwargs)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/www/decorators.py", line 97, in view_func
    return f(*args, **kwargs)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/www/decorators.py", line 60, in wrapper
    return f(*args, **kwargs)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/www/views.py", line 1861, in tree
    dag = current_app.dag_bag.get_dag(dag_id)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/utils/session.py", line 65, in wrapper
    return func(*args, session=session, **kwargs)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/models/dagbag.py", line 171, in get_dag
    self._add_dag_from_db(dag_id=dag_id, session=session)
  File "/root/airflow/2.0.0/venv/lib/python3.8/site-packages/airflow/models/dagbag.py", line 227, in _add_dag_from_db
    raise SerializedDagNotFound(f"DAG '{dag_id}' not found in serialized_dag table")
airflow.exceptions.SerializedDagNotFound: DAG '60040d7f94fe6dd7d7c8a95b' not found in serialized_dag table

60+ dags are dynamically generated from a single file.

I had to patch two places in the scheduler_job.py file where scheduler were keep dying from, then inspecting the dag from the web is throwing the above error.

Here is the patch I applied -

69d68
< from airflow.exceptions import SerializedDagNotFound
1558,1563c1557
<             try:
<                 dag = self.dagbag.get_dag(dag_model.dag_id, session=session)
<             except SerializedDagNotFound as e:
<                 self.log.exception(e)
<                 continue
<
---
>             dag = self.dagbag.get_dag(dag_model.dag_id, session=session)
1601,1606c1595
<             try:
<                 dag = self.dagbag.get_dag(dag_model.dag_id, session=session)
<             except SerializedDagNotFound as e:
<                 self.log.exception(e)
<                 continue
<
---
>             dag = self.dagbag.get_dag(dag_model.dag_id, session=session)

@kaxil kaxil self-assigned this Jan 18, 2021
@nik-davis
Copy link

Would just like to add our temporary solution that is helping us get around this issue, and seems to be working quite nicely. We've added a python script to run before starting the scheduler which will serialize any missing DAGs, so if it fails on this error it will be fixed the next time it starts up.

Here's serialize_missing_dags.py:

from airflow.models import DagBag
from airflow.models.serialized_dag import SerializedDagModel

dag_bag = DagBag()

# Check DB for missing serialized DAGs, and add them if missing
for dag_id in dag_bag.dag_ids:
    if not SerializedDagModel.get(dag_id):
        dag = dag_bag.get_dag(dag_id)
        SerializedDagModel.write_dag(dag)

Which we call before starting the scheduler: python serialize_missing_dags.py && exec airflow scheduler

I hope this helps!

@grillorafael
Copy link

I'm having the same issue

@kaxil
Copy link
Member

kaxil commented Jan 22, 2021

Will be fixed for 2.0.1 -- currently aiming to release it in 2nd week of Feb

kaxil added a commit to astronomer/airflow that referenced this issue Jan 25, 2021
closes apache#13504

Currently the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for preocessing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.
@kaxil
Copy link
Member

kaxil commented Jan 25, 2021

Can one of you try the solution mentioned in #13893 please? @grillorafael @nik-davis @iameugenejo @adamtay82

And also provide a reproducible script (the one that generates dynamic DAGs).

I will add another commit to that PR or a new PR so that Scheduler should be able to handle such cases too.

kaxil added a commit that referenced this issue Jan 25, 2021
closes #13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.
kaxil added a commit that referenced this issue Jan 27, 2021
closes #13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

(cherry picked from commit b9eb51a)
kaxil added a commit that referenced this issue Feb 4, 2021
closes #13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

(cherry picked from commit b9eb51a)
kaxil added a commit to astronomer/airflow that referenced this issue Feb 16, 2021
closes apache#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

(cherry picked from commit b9eb51a)
(cherry picked from commit 253d20a)
@shroffrushabh
Copy link

shroffrushabh commented Mar 8, 2021

Screenshot 2021-03-08 at 5 52 06 PM

Hey @kaxil , I am still seeing the same issue in airflow 2.0.1. Any idea how I can fix this? My scheduler also goes into an unhealthy state after I see this error message.

@kaxil
Copy link
Member

kaxil commented Mar 8, 2021

@shroffrushabh -- Can you post the steps to reproduce your case

@rabsr
Copy link

rabsr commented May 12, 2021

I am still facing the same issue on Airflow 2.0.2. Recently I upgraded Airflow from 2.0.1 to 2.0.2. Getting the same error logs when loading dag.

@kaxil
Copy link
Member

kaxil commented May 12, 2021

I am still facing the same issue on Airflow 2.0.2. Recently I upgraded Airflow from 2.0.1 to 2.0.2. Getting the same error logs when loading dag.

Please raise a different issue with the steps to reproduce

@apache apache locked and limited conversation to collaborators May 12, 2021
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 16, 2021
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

(cherry picked from commit b9eb51a0fb32cd660a5459d73d7323865b34dd99)

GitOrigin-RevId: 253d20ad1cdfd3559daa0ea1f2525bcefb3c1b46
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 17, 2021
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 23, 2021
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 27, 2021
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Mar 10, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jun 4, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jun 7, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

(cherry picked from commit b9eb51a0fb32cd660a5459d73d7323865b34dd99)

GitOrigin-RevId: 253d20ad1cdfd3559daa0ea1f2525bcefb3c1b46
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jul 9, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Aug 27, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Oct 4, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
aglipska pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Oct 7, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Dec 7, 2022
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jan 27, 2023
closes apache/airflow#13504

Currently, the DagFileProcessor parses the DAG files, writes it to the
`dag` table and then writes DAGs to `serialized_dag` table.

At the same time, the scheduler loop is constantly looking for the next
DAGs to process based on ``next_dagrun_create_after`` column of the DAG
table.

It might happen that as soon as the DagFileProcessor writes DAG to `dag`
table, the scheduling loop in the Scheduler picks up the DAG for processing.
However, as the DagFileProcessor has not written to serialized DAG table yet
the scheduler will error with "Serialized Dag not Found" error.

This would mainly happen when the DAGs are dynamic where the result of one DAG,
creates multiple DAGs.

This commit changes the order of writing DAG and Serialized DAG and hence
before a DAG is written to `dag` table it will be written to `serialized_dag` table.

GitOrigin-RevId: b9eb51a0fb32cd660a5459d73d7323865b34dd99
Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.
Labels
affected_version:2.0 Issues Reported for 2.0 area:serialization kind:bug This is a clearly a bug priority:medium Bug that should be fixed before next release but would not block a release
Projects
None yet
Development

Successfully merging a pull request may close this issue.