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

[AIRFLOW-5870] Allow -1 for infinite pool size #6520

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
20 changes: 12 additions & 8 deletions airflow/models/pool.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
# specific language governing permissions and limitations
# under the License.

from sqlalchemy import Column, Integer, String, Text
from sqlalchemy import Column, Integer, String, Text, func

from airflow.models.base import Base
from airflow.ti_deps.deps.pool_slots_available_dep import STATES_TO_COUNT_AS_RUNNING
Expand All @@ -30,6 +30,7 @@ class Pool(Base):

id = Column(Integer, primary_key=True)
pool = Column(String(50), unique=True)
# -1 for infinite
slots = Column(Integer, default=0)
description = Column(Text)

Expand Down Expand Up @@ -64,10 +65,10 @@ def occupied_slots(self, session):
from airflow.models.taskinstance import TaskInstance # Avoid circular import
return (
session
.query(TaskInstance)
.query(func.count())
.filter(TaskInstance.pool == self.pool)
.filter(TaskInstance.state.in_(STATES_TO_COUNT_AS_RUNNING))
.count()
.scalar()
)

@provide_session
Expand All @@ -79,10 +80,10 @@ def used_slots(self, session):

running = (
session
.query(TaskInstance)
.query(func.count())
.filter(TaskInstance.pool == self.pool)
.filter(TaskInstance.state == State.RUNNING)
.count()
.scalar()
)
return running

Expand All @@ -95,15 +96,18 @@ def queued_slots(self, session):

return (
session
.query(TaskInstance)
.query(func.count())
.filter(TaskInstance.pool == self.pool)
.filter(TaskInstance.state == State.QUEUED)
.count()
.scalar()
)

@provide_session
def open_slots(self, session):
"""
Returns the number of slots open at the moment
"""
return self.slots - self.occupied_slots(session)
if self.slots == -1:
return float('inf')
else:
return self.slots - self.occupied_slots(session)
24 changes: 24 additions & 0 deletions tests/models/test_pool.py
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,30 @@ def test_open_slots(self):
self.assertEqual(1, pool.queued_slots())
self.assertEqual(2, pool.occupied_slots())

def test_infinite_slots(self):
pool = Pool(pool='test_pool', slots=-1)
dag = DAG(
dag_id='test_infinite_slots',
start_date=DEFAULT_DATE, )
t1 = DummyOperator(task_id='dummy1', dag=dag, pool='test_pool')
t2 = DummyOperator(task_id='dummy2', dag=dag, pool='test_pool')
ti1 = TI(task=t1, execution_date=DEFAULT_DATE)
ti2 = TI(task=t2, execution_date=DEFAULT_DATE)
ti1.state = State.RUNNING
ti2.state = State.QUEUED

session = settings.Session
session.add(pool)
session.add(ti1)
session.add(ti2)
session.commit()
session.close()

self.assertEqual(float('inf'), pool.open_slots())
self.assertEqual(1, pool.used_slots())
self.assertEqual(1, pool.queued_slots())
self.assertEqual(2, pool.occupied_slots())

def test_default_pool_open_slots(self):
set_default_pool_slots(5)
self.assertEqual(5, Pool.get_default_pool().open_slots())
Expand Down