ashb commented on a change in pull request #10956:
URL: https://github.com/apache/airflow/pull/10956#discussion_r500591552
##########
File path: airflow/models/dag.py
##########
@@ -1824,10 +1960,34 @@ class DagModel(Base):
# Tags for view filter
tags = relationship('DagTag', cascade='all,delete-orphan',
backref=backref('dag'))
+ concurrency = Column(Integer, nullable=False)
+
+ has_task_concurrency_limits = Column(Boolean, nullable=False)
+
+ # The execution_date of the next dag run
+ next_dagrun = Column(UtcDateTime)
+ # Earliest time at which this ``next_dagrun`` can be created
+ next_dagrun_create_after = Column(UtcDateTime)
+
__table_args__ = (
Index('idx_root_dag_id', root_dag_id, unique=False),
+ Index('idx_next_dagrun_create_after', next_dagrun_create_after,
unique=False),
)
+ NUM_DAGS_PER_DAGRUN_QUERY = conf.getint(
+ 'scheduler',
+ 'num_dags_needing_dagrun_per_scheduler_loop',
+ fallback=10
+ )
Review comment:
This is mostly to avoid one scheduler getting all the work of creating
the dag runs, (which can be an "expensive" step right now for large dags)
Description I'm adding to config file is:
```
This changes the number of dags that are locked by each scheduler
when
creating dag runs. One possible reason for setting this lower is if
you
have huge dags and are running multiple schedules, you won't want one
scheduler to do all the work. (Default is 10)
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]