You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/07 19:41:42 UTC

[GitHub] [airflow] kaxil opened a new pull request #11335: Make Dag Serialization a hard requirement

kaxil opened a new pull request #11335:
URL: https://github.com/apache/airflow/pull/11335


   Scheduler HA uses Serialized DAGs and hence it is a strict
   requirement for 2.0.
   
   It also has performance benefits for the Webserver and so should
   be used be default anyway.
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511936395



##########
File path: airflow/api/common/experimental/__init__.py
##########
@@ -32,9 +31,9 @@ def check_and_get_dag(dag_id: str, task_id: Optional[str] = None) -> DagModel:
 
     dagbag = DagBag(
         dag_folder=dag_model.fileloc,
-        read_dags_from_db=conf.getboolean('core', 'store_serialized_dags')
+        read_dags_from_db=True
     )
-    dag = dagbag.get_dag(dag_id)  # prefetch dag if it is stored serialized
+    dag = dagbag.get_dag(dag_id)
     if dag_id not in dagbag.dags:

Review comment:
       Done in 690e80b
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511861198



##########
File path: airflow/api/common/experimental/__init__.py
##########
@@ -32,9 +31,9 @@ def check_and_get_dag(dag_id: str, task_id: Optional[str] = None) -> DagModel:
 
     dagbag = DagBag(
         dag_folder=dag_model.fileloc,
-        read_dags_from_db=conf.getboolean('core', 'store_serialized_dags')
+        read_dags_from_db=True
     )
-    dag = dagbag.get_dag(dag_id)  # prefetch dag if it is stored serialized
+    dag = dagbag.get_dag(dag_id)
     if dag_id not in dagbag.dags:

Review comment:
       ```suggestion
       if not dag:
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] yuqian90 commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
yuqian90 commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r561822542



##########
File path: airflow/models/taskinstance.py
##########
@@ -1576,28 +1574,22 @@ def get(
         }
 
     def get_rendered_template_fields(self):
-        """
-        Fetch rendered template fields from DB if Serialization is enabled.
-        Else just render the templates
-        """
+        """Fetch rendered template fields from DB"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
-        if STORE_SERIALIZED_DAGS:
-            rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
-            if rendered_task_instance_fields:
-                for field_name, rendered_value in rendered_task_instance_fields.items():
-                    setattr(self.task, field_name, rendered_value)
-            else:
-                try:
-                    self.render_templates()
-                except (TemplateAssertionError, UndefinedError) as e:
-                    raise AirflowException(
-                        "Webserver does not have access to User-defined Macros or Filters "
-                        "when Dag Serialization is enabled. Hence for the task that have not yet "
-                        "started running, please use 'airflow tasks render' for debugging the "
-                        "rendering of template_fields."
-                    ) from e
+        rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
+        if rendered_task_instance_fields:
+            for field_name, rendered_value in rendered_task_instance_fields.items():
+                setattr(self.task, field_name, rendered_value)
         else:
-            self.render_templates()
+            try:
+                self.render_templates()
+            except (TemplateAssertionError, UndefinedError) as e:
+                raise AirflowException(
+                    "Webserver does not have access to User-defined Macros or Filters "

Review comment:
       Hi @kaxil , if using Airflow 2.0 or using Airflow 1.10.* with `store_serialized_dags = True` turned on, we hit this error here when user clicks on "Rendered Template" on tasks that are using `user_defined_macros` in jinja template fields.
   
   We also hit a similar problem if user clears `ExternalTaskMarker` that uses `user_defined_macros` in the jinja template fields (because the `dag.clear()` function calls `ti.render_templates()` to figure out the actual values of the template fields. 
   
   How do you recommend addressing these issues going forward? Should we allow some webserver functions to get access to these `user_defined_macros`? Or should we serialize the rendered template values so that webserver can access them?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] yuqian90 commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
yuqian90 commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r561822542



##########
File path: airflow/models/taskinstance.py
##########
@@ -1576,28 +1574,22 @@ def get(
         }
 
     def get_rendered_template_fields(self):
-        """
-        Fetch rendered template fields from DB if Serialization is enabled.
-        Else just render the templates
-        """
+        """Fetch rendered template fields from DB"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
-        if STORE_SERIALIZED_DAGS:
-            rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
-            if rendered_task_instance_fields:
-                for field_name, rendered_value in rendered_task_instance_fields.items():
-                    setattr(self.task, field_name, rendered_value)
-            else:
-                try:
-                    self.render_templates()
-                except (TemplateAssertionError, UndefinedError) as e:
-                    raise AirflowException(
-                        "Webserver does not have access to User-defined Macros or Filters "
-                        "when Dag Serialization is enabled. Hence for the task that have not yet "
-                        "started running, please use 'airflow tasks render' for debugging the "
-                        "rendering of template_fields."
-                    ) from e
+        rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
+        if rendered_task_instance_fields:
+            for field_name, rendered_value in rendered_task_instance_fields.items():
+                setattr(self.task, field_name, rendered_value)
         else:
-            self.render_templates()
+            try:
+                self.render_templates()
+            except (TemplateAssertionError, UndefinedError) as e:
+                raise AirflowException(
+                    "Webserver does not have access to User-defined Macros or Filters "

Review comment:
       Hi @kaxil , if using Airflow 2.0 or using Airflow 1.10.* with `store_serialized_dags = True` turned on, we hit this error here when user clicks on "Rendered Template" on tasks that are using `user_defined_macros` in jinja template fields.
   
   We also hit a similar problem if user clears `ExternalTaskMarker` that uses `user_defined_macros` in the jinja template fields (because the `dag.clear()` function calls `ti.render_templates()` to figure out the actual values of the template fields. 
   
   How do you recommend addressing these issues going forward? Should we allow some webserver functions to get access to these `user_defined_macros`? Or should we serialize the rendered template values so that webserver can access them?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511937795



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -286,9 +287,13 @@ def post_set_task_instances_state(dag_id, session):
     except ValidationError as err:
         raise BadRequest(detail=str(err.messages))
 
-    dag = current_app.dag_bag.get_dag(dag_id)
-    if not dag:
-        error_message = "Dag ID {} not found".format(dag_id)
+    error_message = "Dag ID {} not found".format(dag_id)
+    try:
+        dag = current_app.dag_bag.get_dag(dag_id)
+        if not dag:
+            raise NotFound(error_message)
+    except SerializedDagNotFound:

Review comment:
       I will create a separate issue for it, to address it in a separate PR (not strictly related to this PR), so anyone else can also pick that task too.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r562041231



##########
File path: airflow/models/taskinstance.py
##########
@@ -1576,28 +1574,22 @@ def get(
         }
 
     def get_rendered_template_fields(self):
-        """
-        Fetch rendered template fields from DB if Serialization is enabled.
-        Else just render the templates
-        """
+        """Fetch rendered template fields from DB"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
-        if STORE_SERIALIZED_DAGS:
-            rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
-            if rendered_task_instance_fields:
-                for field_name, rendered_value in rendered_task_instance_fields.items():
-                    setattr(self.task, field_name, rendered_value)
-            else:
-                try:
-                    self.render_templates()
-                except (TemplateAssertionError, UndefinedError) as e:
-                    raise AirflowException(
-                        "Webserver does not have access to User-defined Macros or Filters "
-                        "when Dag Serialization is enabled. Hence for the task that have not yet "
-                        "started running, please use 'airflow tasks render' for debugging the "
-                        "rendering of template_fields."
-                    ) from e
+        rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
+        if rendered_task_instance_fields:
+            for field_name, rendered_value in rendered_task_instance_fields.items():
+                setattr(self.task, field_name, rendered_value)
         else:
-            self.render_templates()
+            try:
+                self.render_templates()
+            except (TemplateAssertionError, UndefinedError) as e:
+                raise AirflowException(
+                    "Webserver does not have access to User-defined Macros or Filters "

Review comment:
       This error is already handled in the Webserver:
   
   https://github.com/apache/airflow/blob/10b8ecc86f24739a38e56347dcc8dc60e3e43975/airflow/www/views.py#L883-L890
   
   i.e we just ask users to run it via a CLI instead.
   
   The `flash` message is so that they know where to look for i.e. run it using the CLI
   
   The `ExternalTaskMarker` does it cause failure in Scheduler or Webserver 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r562041231



##########
File path: airflow/models/taskinstance.py
##########
@@ -1576,28 +1574,22 @@ def get(
         }
 
     def get_rendered_template_fields(self):
-        """
-        Fetch rendered template fields from DB if Serialization is enabled.
-        Else just render the templates
-        """
+        """Fetch rendered template fields from DB"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
-        if STORE_SERIALIZED_DAGS:
-            rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
-            if rendered_task_instance_fields:
-                for field_name, rendered_value in rendered_task_instance_fields.items():
-                    setattr(self.task, field_name, rendered_value)
-            else:
-                try:
-                    self.render_templates()
-                except (TemplateAssertionError, UndefinedError) as e:
-                    raise AirflowException(
-                        "Webserver does not have access to User-defined Macros or Filters "
-                        "when Dag Serialization is enabled. Hence for the task that have not yet "
-                        "started running, please use 'airflow tasks render' for debugging the "
-                        "rendering of template_fields."
-                    ) from e
+        rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
+        if rendered_task_instance_fields:
+            for field_name, rendered_value in rendered_task_instance_fields.items():
+                setattr(self.task, field_name, rendered_value)
         else:
-            self.render_templates()
+            try:
+                self.render_templates()
+            except (TemplateAssertionError, UndefinedError) as e:
+                raise AirflowException(
+                    "Webserver does not have access to User-defined Macros or Filters "

Review comment:
       This error is already handled in the Webserver:
   
   https://github.com/apache/airflow/blob/10b8ecc86f24739a38e56347dcc8dc60e3e43975/airflow/www/views.py#L883-L890
   
   i.e we just ask users to run it via a CLI instead.
   
   The `flash` message is so that they know where to look for i.e. run it using the CLI
   
   The `ExternalTaskMarker` does it cause failure in Scheduler or Webserver 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511457675



##########
File path: tests/jobs/test_scheduler_job.py
##########
@@ -1407,7 +1414,9 @@ def test_find_executable_task_instances_task_concurrency(self):  # pylint: disab
         session = settings.Session()
 
         scheduler.dagbag.bag_dag(dag, root_dag=dag)
-        scheduler.dagbag.sync_to_db(session=session)
+        # Since we don't want to store the code for the DAG defined in this file
+        with mock.patch.object(settings, "STORE_DAG_CODE", False):

Review comment:
       Done in 74850b499




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511863077



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -286,9 +287,13 @@ def post_set_task_instances_state(dag_id, session):
     except ValidationError as err:
         raise BadRequest(detail=str(err.messages))
 
-    dag = current_app.dag_bag.get_dag(dag_id)
-    if not dag:
-        error_message = "Dag ID {} not found".format(dag_id)
+    error_message = "Dag ID {} not found".format(dag_id)
+    try:
+        dag = current_app.dag_bag.get_dag(dag_id)
+        if not dag:
+            raise NotFound(error_message)
+    except SerializedDagNotFound:

Review comment:
       If we made AirflowNotFound inherit from connextion.NotFound this code could be a simpler here.
   
   WDYT?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511936183



##########
File path: airflow/exceptions.py
##########
@@ -126,6 +126,10 @@ class DuplicateTaskIdFound(AirflowException):
     """Raise when a Task with duplicate task_id is defined in the same DAG"""
 
 
+class SerializedDagNotFound(AirflowNotFoundException):

Review comment:
       Done




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511862369



##########
File path: airflow/exceptions.py
##########
@@ -126,6 +126,10 @@ class DuplicateTaskIdFound(AirflowException):
     """Raise when a Task with duplicate task_id is defined in the same DAG"""
 
 
+class SerializedDagNotFound(AirflowNotFoundException):

Review comment:
       ```suggestion
   class SerializedDagNotFound(DagNotFoundException):
   ```
   Maybe?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#issuecomment-715630806


   @ashb This is ready for review


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil merged pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #11335:
URL: https://github.com/apache/airflow/pull/11335


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511450726



##########
File path: tests/jobs/test_scheduler_job.py
##########
@@ -1407,7 +1414,9 @@ def test_find_executable_task_instances_task_concurrency(self):  # pylint: disab
         session = settings.Session()
 
         scheduler.dagbag.bag_dag(dag, root_dag=dag)
-        scheduler.dagbag.sync_to_db(session=session)
+        # Since we don't want to store the code for the DAG defined in this file
+        with mock.patch.object(settings, "STORE_DAG_CODE", False):

Review comment:
       Do we have this on every case? If so we should do it in setup with a second patcher (or even if most we should do it in setup)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#issuecomment-715499053


   [The Workflow run](https://github.com/apache/airflow/actions/runs/324700129) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511936183



##########
File path: airflow/exceptions.py
##########
@@ -126,6 +126,10 @@ class DuplicateTaskIdFound(AirflowException):
     """Raise when a Task with duplicate task_id is defined in the same DAG"""
 
 
+class SerializedDagNotFound(AirflowNotFoundException):

Review comment:
       Done in https://github.com/apache/airflow/pull/11335/commits/690e80b82b12ab1d7232d6f02ea6bddd1d71750c




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r511457675



##########
File path: tests/jobs/test_scheduler_job.py
##########
@@ -1407,7 +1414,9 @@ def test_find_executable_task_instances_task_concurrency(self):  # pylint: disab
         session = settings.Session()
 
         scheduler.dagbag.bag_dag(dag, root_dag=dag)
-        scheduler.dagbag.sync_to_db(session=session)
+        # Since we don't want to store the code for the DAG defined in this file
+        with mock.patch.object(settings, "STORE_DAG_CODE", False):

Review comment:
       Done in 7fcd122




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [airflow] yuqian90 commented on a change in pull request #11335: Make Dag Serialization a hard requirement

Posted by GitBox <gi...@apache.org>.
yuqian90 commented on a change in pull request #11335:
URL: https://github.com/apache/airflow/pull/11335#discussion_r562450598



##########
File path: airflow/models/taskinstance.py
##########
@@ -1576,28 +1574,22 @@ def get(
         }
 
     def get_rendered_template_fields(self):
-        """
-        Fetch rendered template fields from DB if Serialization is enabled.
-        Else just render the templates
-        """
+        """Fetch rendered template fields from DB"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
-        if STORE_SERIALIZED_DAGS:
-            rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
-            if rendered_task_instance_fields:
-                for field_name, rendered_value in rendered_task_instance_fields.items():
-                    setattr(self.task, field_name, rendered_value)
-            else:
-                try:
-                    self.render_templates()
-                except (TemplateAssertionError, UndefinedError) as e:
-                    raise AirflowException(
-                        "Webserver does not have access to User-defined Macros or Filters "
-                        "when Dag Serialization is enabled. Hence for the task that have not yet "
-                        "started running, please use 'airflow tasks render' for debugging the "
-                        "rendering of template_fields."
-                    ) from e
+        rendered_task_instance_fields = RenderedTaskInstanceFields.get_templated_fields(self)
+        if rendered_task_instance_fields:
+            for field_name, rendered_value in rendered_task_instance_fields.items():
+                setattr(self.task, field_name, rendered_value)
         else:
-            self.render_templates()
+            try:
+                self.render_templates()
+            except (TemplateAssertionError, UndefinedError) as e:
+                raise AirflowException(
+                    "Webserver does not have access to User-defined Macros or Filters "

Review comment:
       Hi, @kaxil . Thanks for pointing out that the "Rendered Template" error is handled with an error message. Then that issue is a small annoyance for the user because they cannot see the rendered arguments in the website.
   
   However, the `ExternalTaskMarker` actually causes issue in the Webserver when user hits Clear. I opened an issue here https://github.com/apache/airflow/issues/13827
   
   I don't mind working on a fix. Just wondering if you have any suggestions how to do it.




----------------------------------------------------------------
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:
users@infra.apache.org