Skip to content

Commit

Permalink
Fix accidental hard dependency on dask in BackfillJob (#8025)
Browse files Browse the repository at this point in the history
PR #7786 pulled this back in in such a way that made dask a hard
dependency on `airflow backfill` commands.
  • Loading branch information
ashb authored Mar 31, 2020
1 parent aae3b8f commit 07de602
Showing 1 changed file with 6 additions and 1 deletion.
7 changes: 6 additions & 1 deletion airflow/jobs/backfill_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
AirflowException, DagConcurrencyLimitReached, NoAvailablePoolSlot, PoolNotFound,
TaskConcurrencyLimitReached,
)
from airflow.executors.dask_executor import DaskExecutor
from airflow.executors.local_executor import LocalExecutor
from airflow.executors.sequential_executor import SequentialExecutor
from airflow.jobs.base_job import BaseJob
Expand Down Expand Up @@ -773,6 +772,12 @@ def _execute(self, session=None):

# picklin'
pickle_id = None

try:
from airflow.executors.dask_executor import DaskExecutor
except ImportError:
DaskExecutor = None

if not self.donot_pickle and \
self.executor.__class__ not in (LocalExecutor, SequentialExecutor, DaskExecutor):
pickle = DagPickle(self.dag)
Expand Down

0 comments on commit 07de602

Please sign in to comment.