You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ka...@apache.org on 2021/04/12 11:14:14 UTC

[airflow] branch v2-0-test updated: Unable to trigger backfill or manual jobs with Kubernetes executor. (#14160)

This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a commit to branch v2-0-test
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/v2-0-test by this push:
     new 3634738  Unable to trigger backfill or manual jobs with Kubernetes executor. (#14160)
3634738 is described below

commit 36347385766c55f9cb616256b44a09e37b6cf991
Author: Max Taggart <ma...@gmail.com>
AuthorDate: Thu Feb 25 12:16:55 2021 -0700

    Unable to trigger backfill or manual jobs with Kubernetes executor. (#14160)
    
    closes: #13805
    (cherry picked from commit 2b5d4e3ff3c61ea6074caa300bbb8d16027408a6)
---
 airflow/jobs/backfill_job.py    |  1 +
 airflow/www/views.py            |  1 +
 tests/jobs/test_backfill_job.py | 17 +++++++++++++++++
 3 files changed, 19 insertions(+)

diff --git a/airflow/jobs/backfill_job.py b/airflow/jobs/backfill_job.py
index 0d3d057..a16f261 100644
--- a/airflow/jobs/backfill_job.py
+++ b/airflow/jobs/backfill_job.py
@@ -785,6 +785,7 @@ class BackfillJob(BaseJob):
             pickle_id = pickle.id
 
         executor = self.executor
+        executor.job_id = "backfill"
         executor.start()
 
         ti_status.total_runs = len(run_dates)  # total dag runs in backfill
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 5f4c8c5..a38b9cc 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -1377,6 +1377,7 @@ class Airflow(AirflowBaseView):  # noqa: D101  pylint: disable=too-many-public-m
             )
             return redirect(origin)
 
+        executor.job_id = "manual"
         executor.start()
         executor.queue_task_instance(
             ti,
diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py
index 3139f36..c6f620a 100644
--- a/tests/jobs/test_backfill_job.py
+++ b/tests/jobs/test_backfill_job.py
@@ -1517,3 +1517,20 @@ class TestBackfillJob(unittest.TestCase):
         job.run()
         dr: DagRun = dag.get_last_dagrun()
         assert dr.creating_job_id == job.id
+
+    def test_backfill_has_job_id(self):
+        """Make sure that backfill jobs are assigned job_ids."""
+        dag = self.dagbag.get_dag("test_start_date_scheduling")
+        dag.clear()
+
+        executor = MockExecutor(parallelism=16)
+
+        job = BackfillJob(
+            executor=executor,
+            dag=dag,
+            start_date=DEFAULT_DATE,
+            end_date=DEFAULT_DATE + datetime.timedelta(days=1),
+            run_backwards=True,
+        )
+        job.run()
+        assert executor.job_id is not None