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 2020/06/29 20:43:32 UTC

[airflow] branch v1-10-test updated (915dbea -> f63b249)

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

kaxilnaik pushed a change to branch v1-10-test
in repository https://gitbox.apache.org/repos/asf/airflow.git.


    from 915dbea  Add support for fetching logs from running pods (#8626)
     new a0fc255  Fix the default value for store_dag_code (#9554)
     new 3f1ea4f  Fix failing test in DagCode (#9565)
     new 6804841  fixup! Avoid color info in response of /dag_stats & /task_stats (#8742)
     new f63b249  Allow changing Task States Colors (#9520)

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 airflow/config_templates/config.yml            |  4 ++--
 airflow/config_templates/default_airflow.cfg   |  3 ++-
 airflow/models/dag.py                          |  2 +-
 airflow/models/dagcode.py                      |  4 ++--
 airflow/settings.py                            | 19 ++++++++++++++++++
 airflow/utils/dag_processing.py                |  4 ++--
 airflow/utils/state.py                         |  3 +++
 airflow/www/app.py                             |  5 +++--
 airflow/www/templates/admin/master.html        |  7 +++++++
 airflow/www/templates/airflow/dags.html        |  5 +++--
 airflow/www/templates/airflow/gantt.html       |  7 +++++++
 airflow/www/templates/airflow/graph.html       | 18 +++++++++--------
 airflow/www/templates/airflow/tree.html        | 27 +++++++++++---------------
 airflow/www/views.py                           | 13 ++++++++-----
 airflow/www_rbac/app.py                        |  4 +++-
 airflow/www_rbac/templates/airflow/gantt.html  |  7 +++++++
 airflow/www_rbac/templates/airflow/graph.html  | 18 +++++++++--------
 airflow/www_rbac/templates/airflow/master.html |  8 +++++++-
 airflow/www_rbac/templates/airflow/tree.html   | 27 +++++++++++---------------
 airflow/www_rbac/views.py                      |  3 ++-
 tests/models/test_dagcode.py                   |  6 ++++++
 tests/test_configuration.py                    | 20 +++++++++++++++++++
 22 files changed, 146 insertions(+), 68 deletions(-)


[airflow] 03/04: fixup! Avoid color info in response of /dag_stats & /task_stats (#8742)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6804841948b11fb343733fb098dddebbf2b30e72
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Mon Jun 29 19:53:38 2020 +0100

    fixup! Avoid color info in response of /dag_stats & /task_stats (#8742)
---
 airflow/www/templates/airflow/dags.html |  5 +++--
 airflow/www/views.py                    | 10 ++++++----
 2 files changed, 9 insertions(+), 6 deletions(-)

diff --git a/airflow/www/templates/airflow/dags.html b/airflow/www/templates/airflow/dags.html
index ec4e4d0..6d7019d 100644
--- a/airflow/www/templates/airflow/dags.html
+++ b/airflow/www/templates/airflow/dags.html
@@ -217,6 +217,7 @@
 
       const DAGS_INDEX = "{{ url_for('admin.index') }}";
       const ENTER_KEY_CODE = 13;
+      const STATE_COLOR = {{ state_color|tojson }};
 
       $('#dag_query').on('keypress', function (e) {
         // check for key press on ENTER (key code 13) to trigger the search
@@ -349,7 +350,7 @@
               })
               .attr('stroke', function(d) {
                   if (d.count > 0)
-                    return d.color;
+                    return STATE_COLOR[d.state];
                   else {
                     return 'gainsboro';
                   }
@@ -428,7 +429,7 @@
               })
               .attr('stroke', function(d) {
                   if (d.count > 0)
-                    return d.color;
+                    return STATE_COLOR[d.state];
                   else {
                     return 'gainsboro';
                   }
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 9d8e37a..737b1e4 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -602,8 +602,7 @@ class Airflow(AirflowViewMixin, BaseView):
                 count = d.get(state, 0)
                 payload[dag_id].append({
                     'state': state,
-                    'count': count,
-                    'color': State.color(state)
+                    'count': count
                 })
         return wwwutils.json_response(payload)
 
@@ -685,8 +684,7 @@ class Airflow(AirflowViewMixin, BaseView):
                 count = data.get(dag_id, {}).get(state, 0)
                 payload[dag_id].append({
                     'state': state,
-                    'count': count,
-                    'color': State.color(state)
+                    'count': count
                 })
         return wwwutils.json_response(payload)
 
@@ -2319,11 +2317,15 @@ class HomeView(AirflowViewMixin, AdminIndexView):
             auto_complete_data.add(row.dag_id)
             auto_complete_data.add(row.owners)
 
+        state_color_mapping = State.state_color.copy()
+        state_color_mapping["null"] = state_color_mapping.pop(None)
+
         return self.render(
             'airflow/dags.html',
             dags=dags,
             hide_paused=hide_paused,
             current_page=current_page,
+            state_color=state_color_mapping,
             search_query=arg_search_query if arg_search_query else '',
             page_size=dags_per_page,
             num_of_pages=num_of_pages,


[airflow] 02/04: Fix failing test in DagCode (#9565)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3f1ea4f0cec1cb00b01e583bc469f88a387bb39a
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Mon Jun 29 15:16:41 2020 +0100

    Fix failing test in DagCode (#9565)
    
    PR https://github.com/apache/airflow/pull/9554 introduced this error and because of Github issue currently (github is down / has degraded performance) the CI didn't run fully
    
    (cherry picked from commit ee0335315e421c8ce9e826f7ac5e8f4c82f171fe)
---
 tests/models/test_dagcode.py | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/tests/models/test_dagcode.py b/tests/models/test_dagcode.py
index 3eb5f34..2926de6 100644
--- a/tests/models/test_dagcode.py
+++ b/tests/models/test_dagcode.py
@@ -56,6 +56,7 @@ class TestDagCode(unittest.TestCase):
         return [bash_dag, xcom_dag]
 
     @conf_vars({('core', 'store_dag_code'): 'True'})
+    @patch("airflow.models.dag.settings.STORE_DAG_CODE", True)
     def _write_example_dags(self):
         example_dags = make_example_dags(example_dags_module)
         for dag in example_dags.values():
@@ -68,6 +69,7 @@ class TestDagCode(unittest.TestCase):
 
         self._compare_example_dags(example_dags)
 
+    @conf_vars({('core', 'store_dag_code'): 'True'})
     def test_bulk_sync_to_db(self):
         """Dg code can be bulk written into database."""
         example_dags = make_example_dags(example_dags_module)
@@ -78,6 +80,7 @@ class TestDagCode(unittest.TestCase):
 
         self._compare_example_dags(example_dags)
 
+    @conf_vars({('core', 'store_dag_code'): 'True'})
     def test_bulk_sync_to_db_half_files(self):
         """Dg code can be bulk written into database."""
         example_dags = make_example_dags(example_dags_module)
@@ -119,6 +122,8 @@ class TestDagCode(unittest.TestCase):
                 self.assertEqual(result.source_code, source_code)
 
     @conf_vars({('core', 'store_dag_code'): 'True'})
+    @patch("airflow.models.dag.settings.STORE_DAG_CODE", True)
+    @patch("airflow.models.dagcode.STORE_DAG_CODE", True)
     def test_code_can_be_read_when_no_access_to_file(self):
         """
         Test that code can be retrieved from DB when you do not have access to Code file.
@@ -140,6 +145,7 @@ class TestDagCode(unittest.TestCase):
                 self.assertIn(test_string, dag_code)
 
     @conf_vars({('core', 'store_dag_code'): 'True'})
+    @patch("airflow.models.dag.settings.STORE_DAG_CODE", True)
     def test_db_code_updated_on_dag_file_change(self):
         """Test if DagCode is updated in DB when DAG file is changed"""
         example_dag = make_example_dags(example_dags_module).get('example_bash_operator')


[airflow] 01/04: Fix the default value for store_dag_code (#9554)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a0fc25593925ab33c8a147ecc0fa987f043353ec
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Mon Jun 29 12:46:06 2020 +0100

    Fix the default value for store_dag_code (#9554)
    
    related to #8255 (fixes the issue mentioned with `store_dag_code` but does not address Config interpolation)
    
    The default value of `store_dag_code` should be same as `store_serialized_dags` setting.  But if the value is set it should use that value
    
    (cherry picked from commit 57c722b65c1ddfe527924448291f29ff7036ad0a)
---
 airflow/config_templates/config.yml          |  4 ++--
 airflow/config_templates/default_airflow.cfg |  3 ++-
 airflow/models/dag.py                        |  2 +-
 airflow/models/dagcode.py                    |  4 ++--
 airflow/settings.py                          |  5 +++++
 airflow/utils/dag_processing.py              |  4 ++--
 tests/test_configuration.py                  | 20 ++++++++++++++++++++
 7 files changed, 34 insertions(+), 8 deletions(-)

diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml
index 61491d8..e32b8cc 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -445,8 +445,8 @@
         ``store_serialized_dags`` setting.
       version_added: 1.10.10
       type: string
-      example: ~
-      default: "%(store_serialized_dags)s"
+      example: "False"
+      default: ~
     - name: max_num_rendered_ti_fields_per_task
       description: |
         Maximum number of Rendered Task Instance Fields (Template Fields) per task to store
diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg
index 2cc97e2..c75d3ae 100644
--- a/airflow/config_templates/default_airflow.cfg
+++ b/airflow/config_templates/default_airflow.cfg
@@ -232,7 +232,8 @@ min_serialized_dag_update_interval = 30
 # If set to True, Webserver reads file contents from DB instead of
 # trying to access files in a DAG folder. Defaults to same as the
 # ``store_serialized_dags`` setting.
-store_dag_code = %(store_serialized_dags)s
+# Example: store_dag_code = False
+# store_dag_code =
 
 # Maximum number of Rendered Task Instance Fields (Template Fields) per task to store
 # in the Database.
diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index 7759cb3..933dc10 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -1529,7 +1529,7 @@ class DAG(BaseDag, LoggingMixin):
         orm_dag.schedule_interval = self.schedule_interval
         orm_dag.tags = self.get_dagtags(session=session)
 
-        if conf.getboolean('core', 'store_dag_code', fallback=False):
+        if settings.STORE_DAG_CODE:
             DagCode.bulk_sync_to_db([orm_dag.fileloc])
 
         session.commit()
diff --git a/airflow/models/dagcode.py b/airflow/models/dagcode.py
index 513ec18..6aa7b6a 100644
--- a/airflow/models/dagcode.py
+++ b/airflow/models/dagcode.py
@@ -21,9 +21,9 @@ from datetime import datetime
 
 from sqlalchemy import BigInteger, Column, String, UnicodeText, and_, exists
 
-from airflow.configuration import conf
 from airflow.exceptions import AirflowException, DagCodeNotFound
 from airflow.models import Base
+from airflow.settings import STORE_DAG_CODE
 from airflow.utils import timezone
 from airflow.utils.file import correct_maybe_zipped, open_maybe_zipped
 from airflow.utils.db import provide_session
@@ -178,7 +178,7 @@ class DagCode(Base):
 
         :return: source code as string
         """
-        if conf.getboolean('core', 'store_dag_code', fallback=False):
+        if STORE_DAG_CODE:
             return cls._get_code_from_db(fileloc)
         else:
             return cls._get_code_from_file(fileloc)
diff --git a/airflow/settings.py b/airflow/settings.py
index 513f192..c86d4b2 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -401,6 +401,11 @@ STORE_SERIALIZED_DAGS = conf.getboolean('core', 'store_serialized_dags', fallbac
 MIN_SERIALIZED_DAG_UPDATE_INTERVAL = conf.getint(
     'core', 'min_serialized_dag_update_interval', fallback=30)
 
+# Whether to persist DAG files code in DB. If set to True, Webserver reads file contents
+# from DB instead of trying to access files in a DAG folder.
+# Defaults to same as the store_serialized_dags setting.
+STORE_DAG_CODE = conf.getboolean("core", "store_dag_code", fallback=STORE_SERIALIZED_DAGS)
+
 # If donot_modify_handlers=True, we do not modify logging handlers in task_run command
 # If the flag is set to False, we remove all handlers from the root logger
 # and add all handlers from 'airflow.task' logger to the root Logger. This is done
diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py
index 6e4e045..3aac8fd 100644
--- a/airflow/utils/dag_processing.py
+++ b/airflow/utils/dag_processing.py
@@ -50,7 +50,7 @@ from airflow.dag.base_dag import BaseDag, BaseDagBag
 from airflow.exceptions import AirflowException
 from airflow.settings import Stats
 from airflow.models import errors
-from airflow.settings import STORE_SERIALIZED_DAGS
+from airflow.settings import STORE_DAG_CODE, STORE_SERIALIZED_DAGS
 from airflow.utils import timezone
 from airflow.utils.helpers import reap_process_group
 from airflow.utils.db import provide_session
@@ -914,7 +914,7 @@ class DagFileProcessorManager(LoggingMixin):
                 SerializedDagModel.remove_deleted_dags(self._file_paths)
                 DagModel.deactivate_deleted_dags(self._file_paths)
 
-            if conf.getboolean('core', 'store_dag_code', fallback=False):
+            if STORE_DAG_CODE:
                 from airflow.models.dagcode import DagCode
                 DagCode.remove_deleted_code(self._file_paths)
 
diff --git a/tests/test_configuration.py b/tests/test_configuration.py
index af1df69..5c40cad 100644
--- a/tests/test_configuration.py
+++ b/tests/test_configuration.py
@@ -32,6 +32,7 @@ import six
 from airflow import configuration
 from airflow.configuration import conf, AirflowConfigParser, parameterized_config
 from tests.compat import mock
+from tests.test_utils.config import conf_vars
 from tests.test_utils.reset_warning_registry import reset_warning_registry
 
 if six.PY2:
@@ -494,3 +495,22 @@ notacommand = OK
             conf.write(string_file)
             content = string_file.getvalue()
         self.assertIn("dags_folder = /tmp/test_folder", content)
+
+    @conf_vars({("core", "store_serialized_dags"): "True"})
+    def test_store_dag_code_default_config(self):
+        store_serialized_dags = conf.getboolean('core', 'store_serialized_dags', fallback=False)
+        store_dag_code = conf.getboolean("core", "store_dag_code", fallback=store_serialized_dags)
+        self.assertFalse(conf.has_option("core", "store_dag_code"))
+        self.assertTrue(store_serialized_dags)
+        self.assertTrue(store_dag_code)
+
+    @conf_vars({
+        ("core", "store_serialized_dags"): "True",
+        ("core", "store_dag_code"): "False"
+    })
+    def test_store_dag_code_config_when_set(self):
+        store_serialized_dags = conf.getboolean('core', 'store_serialized_dags', fallback=False)
+        store_dag_code = conf.getboolean("core", "store_dag_code", fallback=store_serialized_dags)
+        self.assertTrue(conf.has_option("core", "store_dag_code"))
+        self.assertTrue(store_serialized_dags)
+        self.assertFalse(store_dag_code)


[airflow] 04/04: Allow changing Task States Colors (#9520)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f63b2496ccd0c78e3e918f59f473012c834967f1
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Mon Jun 29 16:11:24 2020 +0100

    Allow changing Task States Colors (#9520)
    
    (cherry picked from commit e1108d42d2b3b9c049ed8872176923621b4c8d79)
---
 airflow/settings.py                            | 14 +++++++++++++
 airflow/utils/state.py                         |  3 +++
 airflow/www/app.py                             |  5 +++--
 airflow/www/templates/admin/master.html        |  7 +++++++
 airflow/www/templates/airflow/gantt.html       |  7 +++++++
 airflow/www/templates/airflow/graph.html       | 18 +++++++++--------
 airflow/www/templates/airflow/tree.html        | 27 +++++++++++---------------
 airflow/www/views.py                           |  3 ++-
 airflow/www_rbac/app.py                        |  4 +++-
 airflow/www_rbac/templates/airflow/gantt.html  |  7 +++++++
 airflow/www_rbac/templates/airflow/graph.html  | 18 +++++++++--------
 airflow/www_rbac/templates/airflow/master.html |  8 +++++++-
 airflow/www_rbac/templates/airflow/tree.html   | 27 +++++++++++---------------
 airflow/www_rbac/views.py                      |  3 ++-
 14 files changed, 97 insertions(+), 54 deletions(-)

diff --git a/airflow/settings.py b/airflow/settings.py
index c86d4b2..c56c3a8 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -153,6 +153,20 @@ Session = None
 # The JSON library to use for DAG Serialization and De-Serialization
 json = json
 
+# Dictionary containing State and colors associated to each state to
+# display on the Webserver
+STATE_COLORS = {
+    "queued": "gray",
+    "running": "lime",
+    "success": "green",
+    "failed": "red",
+    "up_for_retry": "gold",
+    "up_for_reschedule": "turquoise",
+    "upstream_failed": "orange",
+    "skipped": "pink",
+    "scheduled": "tan",
+}
+
 
 def policy(task_instance):
     """
diff --git a/airflow/utils/state.py b/airflow/utils/state.py
index 320b996..bb3ad39 100644
--- a/airflow/utils/state.py
+++ b/airflow/utils/state.py
@@ -21,6 +21,8 @@ from __future__ import unicode_literals
 
 from builtins import object
 
+from airflow.settings import STATE_COLORS
+
 
 class State(object):
     """
@@ -80,6 +82,7 @@ class State(object):
         SCHEDULED: 'tan',
         NONE: 'lightblue',
     }
+    state_color.update(STATE_COLORS)  # type: ignore
 
     @classmethod
     def color(cls, state):
diff --git a/airflow/www/app.py b/airflow/www/app.py
index c5c6066..30b9f75 100644
--- a/airflow/www/app.py
+++ b/airflow/www/app.py
@@ -32,7 +32,7 @@ import airflow
 from airflow import models, version, LoggingMixin
 from airflow.configuration import conf
 from airflow.models.connection import Connection
-from airflow.settings import Session
+from airflow.settings import Session, STATE_COLORS
 
 from airflow.www.blueprints import routes
 from airflow.logging_config import configure_logging
@@ -188,7 +188,8 @@ def create_app(config=None, testing=False):
                 'log_auto_tailing_offset': conf.getint(
                     'webserver', 'log_auto_tailing_offset', fallback=30),
                 'log_animation_speed': conf.getint(
-                    'webserver', 'log_animation_speed', fallback=1000)
+                    'webserver', 'log_animation_speed', fallback=1000),
+                'state_color_mapping': STATE_COLORS
             }
 
         @app.before_request
diff --git a/airflow/www/templates/admin/master.html b/airflow/www/templates/admin/master.html
index 531fac0..4300972 100644
--- a/airflow/www/templates/admin/master.html
+++ b/airflow/www/templates/admin/master.html
@@ -23,6 +23,13 @@
   <link href="{{ url_for('static', filename='bootstrap-theme.css') }}" rel="stylesheet">
   <link rel="icon" type="image/png" href="{{ url_for("static", filename="pin_32.png") }}">
   <link rel="stylesheet" type="text/css" href="{{ url_for("static", filename="main.css") }}">
+    <style type="text/css">
+    {% for state, state_color in state_color_mapping.items() %}
+      span.{{state}} {
+        background-color: {{state_color}};
+      }
+    {% endfor %}
+  </style>
 {% endblock %}
 
 {% block tail_js %}
diff --git a/airflow/www/templates/airflow/gantt.html b/airflow/www/templates/airflow/gantt.html
index f2bea89..1889853 100644
--- a/airflow/www/templates/airflow/gantt.html
+++ b/airflow/www/templates/airflow/gantt.html
@@ -24,6 +24,13 @@
 <link href="{{ admin_static.url(filename='vendor/bootstrap-daterangepicker/daterangepicker-bs2.css') }}" rel="stylesheet"/>
 <link type="text/css" href="{{ url_for('static', filename='gantt.css') }}" rel="stylesheet" />
 <link type="text/css" href="{{ url_for('static', filename='tree.css') }}" rel="stylesheet" />
+<style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    rect.{{state}} {
+      fill: {{state_color}};
+    }
+  {% endfor %}
+</style>
 {% endblock %}
 
 {% block body %}
diff --git a/airflow/www/templates/airflow/graph.html b/airflow/www/templates/airflow/graph.html
index dcbe4f9..880c463 100644
--- a/airflow/www/templates/airflow/graph.html
+++ b/airflow/www/templates/airflow/graph.html
@@ -27,6 +27,13 @@
 {{ super() }}
 <link rel="stylesheet" type="text/css" href="{{ url_for('static', filename='dagre.css') }}">
 <link rel="stylesheet" type="text/css" href="{{ url_for('static', filename='graph.css') }}">
+<style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    g.node.{{state}} rect {
+      stroke: {{state_color}};
+    }
+  {% endfor %}
+</style>
 {% endblock %}
 
 {% block body %}
@@ -64,14 +71,9 @@
 
   <div>
     <div class="legend_item state" style="border-color:white;">no_status</div>
-    <div class="legend_item state" style="border-color:grey;">queued</div>
-    <div class="legend_item state" style="border-color:gold;">up_for_retry</div>
-    <div class="legend_item state" style="border-color:turquoise;">up_for_reschedule</div>
-    <div class="legend_item state" style="border-color:orange;">upstream_failed</div>
-    <div class="legend_item state" style="border-color:pink;">skipped</div>
-    <div class="legend_item state" style="border-color:red;">failed</div>
-    <div class="legend_item state" style="border-color:lime;">running</div>
-    <div class="legend_item state" style="border-color:green;">success</div>
+    {% for state, state_color in state_color_mapping.items() %}
+      <div class="legend_item state" style="border-color:{{state_color}};">{{state}}</div>
+    {% endfor %}
   </div>
   <div style="clear:both;"></div>
 </div>
diff --git a/airflow/www/templates/airflow/tree.html b/airflow/www/templates/airflow/tree.html
index 66255f6..cb43ae9 100644
--- a/airflow/www/templates/airflow/tree.html
+++ b/airflow/www/templates/airflow/tree.html
@@ -24,6 +24,13 @@
 {{ super() }}
 <link rel="stylesheet" type="text/css" href="{{ url_for('static', filename='tree.css') }}">
 <link href="{{ admin_static.url(filename='vendor/bootstrap-daterangepicker/daterangepicker-bs2.css') }}" rel="stylesheet">
+<style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    rect.{{state}} {
+      fill: {{state_color}};
+    }
+  {% endfor %}
+</style>
 {% endblock %}
 
 {% block body %}
@@ -46,22 +53,10 @@
 <div>
   <div class="legend_item" style="border: none;">no_status</div>
   <div class="square" style="background: white;"></div>
-  <div class="legend_item" style="border: none;">queued</div>
-  <div class="square" style="background: grey;"></div>
-  <div class="legend_item" style="border: none;">up_for_retry</div>
-  <div class="square" style="background: gold;"></div>
-  <div class="legend_item" style="border: none;">up_for_reschedule</div>
-  <div class="square" style="background: turquoise;"></div>
-  <div class="legend_item" style="border: none;">upstream_failed</div>
-  <div class="square" style="background: orange;"></div>
-  <div class="legend_item" style="border: none;">skipped</div>
-  <div class="square" style="background: pink;"></div>
-  <div class="legend_item" style="border: none;">failed</div>
-  <div class="square" style="background: red;"></div>
-  <div class="legend_item" style="border: none;">running</div>
-  <div class="square" style="background: lime;"></div>
-  <div class="legend_item" style="border: none;">success</div>
-  <div class="square" style="background: green;"></div>
+  {% for state, state_color in state_color_mapping.items() %}
+    <div class="legend_item" style="border: none;">{{state}}</div>
+    <div class="square" style="background: {{state_color}};"></div>
+  {% endfor %}
   {% for op in operators %}
   <div class="legend_circle" style="background:{{ op.ui_color }};">
   </div>
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 737b1e4..5a76d8b 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -75,7 +75,7 @@ from airflow.api.common.experimental.mark_tasks import (set_dag_run_state_to_run
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator, Connection, DagRun, errors, XCom
 from airflow.models.dagcode import DagCode
-from airflow.settings import STORE_SERIALIZED_DAGS
+from airflow.settings import STATE_COLORS, STORE_SERIALIZED_DAGS
 from airflow.operators.subdag_operator import SubDagOperator
 from airflow.ti_deps.dep_context import RUNNING_DEPS, SCHEDULER_QUEUED_DEPS, DepContext
 from airflow.utils import timezone
@@ -2319,6 +2319,7 @@ class HomeView(AirflowViewMixin, AdminIndexView):
 
         state_color_mapping = State.state_color.copy()
         state_color_mapping["null"] = state_color_mapping.pop(None)
+        state_color_mapping.update(STATE_COLORS)
 
         return self.render(
             'airflow/dags.html',
diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py
index ca6c4c8..8d59c85 100644
--- a/airflow/www_rbac/app.py
+++ b/airflow/www_rbac/app.py
@@ -35,6 +35,7 @@ from werkzeug.middleware.dispatcher import DispatcherMiddleware
 from airflow import settings, version
 from airflow.configuration import conf
 from airflow.logging_config import configure_logging
+from airflow.settings import STATE_COLORS
 from airflow.www_rbac.static_config import configure_manifest_files
 
 app = None  # type: Any
@@ -242,7 +243,8 @@ def create_app(config=None, session=None, testing=False, app_name="Airflow"):
                 'log_auto_tailing_offset': conf.getint(
                     'webserver', 'log_auto_tailing_offset', fallback=30),
                 'log_animation_speed': conf.getint(
-                    'webserver', 'log_animation_speed', fallback=1000)
+                    'webserver', 'log_animation_speed', fallback=1000),
+                'state_color_mapping': STATE_COLORS
             }
 
             if 'analytics_tool' in conf.getsection('webserver'):
diff --git a/airflow/www_rbac/templates/airflow/gantt.html b/airflow/www_rbac/templates/airflow/gantt.html
index 7a5e954..c5c4c42 100644
--- a/airflow/www_rbac/templates/airflow/gantt.html
+++ b/airflow/www_rbac/templates/airflow/gantt.html
@@ -21,6 +21,13 @@
 {{ super() }}
 <link type="text/css" href="{{ url_for('static', filename='css/gantt.css') }}" rel="stylesheet" />
 <link type="text/css" href="{{ url_for('static', filename='css/tree.css') }}" rel="stylesheet" />
+<style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    rect.{{state}} {
+      fill: {{state_color}};
+    }
+  {% endfor %}
+</style>
 {% endblock %}
 
 {% block content %}
diff --git a/airflow/www_rbac/templates/airflow/graph.html b/airflow/www_rbac/templates/airflow/graph.html
index e2ab0f1..5fe326e 100644
--- a/airflow/www_rbac/templates/airflow/graph.html
+++ b/airflow/www_rbac/templates/airflow/graph.html
@@ -22,6 +22,13 @@
 {% block head_css %}
 {{ super() }}
 <link rel="stylesheet" type="text/css" href="{{ url_for('static', filename='css/graph.css') }}">
+<style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    g.node.{{state}} rect {
+      stroke: {{state_color}};
+    }
+  {% endfor %}
+</style>
 {% endblock %}
 
 {% block content %}
@@ -58,14 +65,9 @@
 
   <div>
     <div class="legend_item state" style="border-color:white;">no_status</div>
-    <div class="legend_item state" style="border-color:grey;">queued</div>
-    <div class="legend_item state" style="border-color:gold;">up_for_retry</div>
-    <div class="legend_item state" style="border-color:turquoise;">up_for_reschedule</div>
-    <div class="legend_item state" style="border-color:orange;">upstream_failed</div>
-    <div class="legend_item state" style="border-color:pink;">skipped</div>
-    <div class="legend_item state" style="border-color:red;">failed</div>
-    <div class="legend_item state" style="border-color:lime;">running</div>
-    <div class="legend_item state" style="border-color:green;">success</div>
+  {% for state, state_color in state_color_mapping.items() %}
+    <div class="legend_item state" style="border-color:{{state_color}};">{{state}}</div>
+  {% endfor %}
   </div>
   <div style="clear:both;"></div>
 </div>
diff --git a/airflow/www_rbac/templates/airflow/master.html b/airflow/www_rbac/templates/airflow/master.html
index eb30303..fb2d382 100644
--- a/airflow/www_rbac/templates/airflow/master.html
+++ b/airflow/www_rbac/templates/airflow/master.html
@@ -27,7 +27,13 @@
   {% endif %}
   <link href="{{ url_for_asset('main.css') }}" rel="stylesheet">
   <link href="{{ url_for_asset('bootstrap-datetimepicker.min.css') }}" rel="stylesheet">
-
+  <style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    span.{{state}} {
+      background-color: {{state_color}};
+    }
+  {% endfor %}
+  </style>
   <link rel="icon" type="image/png" href="{{ url_for('static', filename='pin_32.png') }}">
 {% endblock %}
 
diff --git a/airflow/www_rbac/templates/airflow/tree.html b/airflow/www_rbac/templates/airflow/tree.html
index b8d167c..9d05a53 100644
--- a/airflow/www_rbac/templates/airflow/tree.html
+++ b/airflow/www_rbac/templates/airflow/tree.html
@@ -21,6 +21,13 @@
 {% block head_css %}
 {{ super() }}
 <link rel="stylesheet" type="text/css" href="{{ url_for('static', filename='css/tree.css') }}">
+<style type="text/css">
+  {% for state, state_color in state_color_mapping.items() %}
+    rect.{{state}} {
+      fill: {{state_color}};
+    }
+  {% endfor %}
+</style>
 {% endblock %}
 
 {% block content %}
@@ -44,22 +51,10 @@
 <div>
   <div class="legend_item" style="border: none;">no_status</div>
   <div class="square" style="background: white;"></div>
-  <div class="legend_item" style="border: none;">queued</div>
-  <div class="square" style="background: grey;"></div>
-  <div class="legend_item" style="border: none;">up_for_retry</div>
-  <div class="square" style="background: gold;"></div>
-  <div class="legend_item" style="border: none;">up_for_reschedule</div>
-  <div class="square" style="background: turquoise;"></div>
-  <div class="legend_item" style="border: none;">upstream_failed</div>
-  <div class="square" style="background: orange;"></div>
-  <div class="legend_item" style="border: none;">skipped</div>
-  <div class="square" style="background: pink;"></div>
-  <div class="legend_item" style="border: none;">failed</div>
-  <div class="square" style="background: red;"></div>
-  <div class="legend_item" style="border: none;">running</div>
-  <div class="square" style="background: lime;"></div>
-  <div class="legend_item" style="border: none;">success</div>
-  <div class="square" style="background: green;"></div>
+  {% for state, state_color in state_color_mapping.items() %}
+    <div class="legend_item" style="border: none;">{{state}}</div>
+    <div class="square" style="background: {{state_color}};"></div>
+  {% endfor %}
   {% for op in operators %}
   <div class="legend_circle" style="background:{{ op.ui_color }};">
   </div>
diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py
index a626d87..326b635 100644
--- a/airflow/www_rbac/views.py
+++ b/airflow/www_rbac/views.py
@@ -63,7 +63,7 @@ from airflow.api.common.experimental.mark_tasks import (set_dag_run_state_to_suc
 from airflow.models import Connection, DagModel, DagRun, DagTag, Log, SlaMiss, TaskFail, XCom, errors
 from airflow.exceptions import AirflowException
 from airflow.models.dagcode import DagCode
-from airflow.settings import STORE_SERIALIZED_DAGS
+from airflow.settings import STATE_COLORS, STORE_SERIALIZED_DAGS
 from airflow.ti_deps.dep_context import RUNNING_DEPS, SCHEDULER_QUEUED_DEPS, DepContext
 from airflow.utils import timezone
 from airflow.utils.dates import infer_time_unit, scale_time_units
@@ -334,6 +334,7 @@ class Airflow(AirflowBaseView):
 
         state_color_mapping = State.state_color.copy()
         state_color_mapping["null"] = state_color_mapping.pop(None)
+        state_color_mapping.update(STATE_COLORS)
 
         return self.render_template(
             'airflow/dags.html',