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 2022/12/19 22:01:50 UTC

[GitHub] [airflow] michaelmicheal opened a new pull request, #28481: dagwarning, add error handling to purge_inactive_dag_warnings

michaelmicheal opened a new pull request, #28481:
URL: https://github.com/apache/airflow/pull/28481

   This PR safely catches exceptions to the `purge_inactive_dag_warnings`. This prevents unhandled exceptions from causing the DagFileProcessorManager to fail.
   related: #28480 
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+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 a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1059236012


##########
tests/dag_warning/test_dag_warning.py:
##########
@@ -0,0 +1,52 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+import unittest
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models.dagwarning import DagWarning
+
+
+class TestDagWarning(unittest.TestCase):

Review Comment:
   By convention, this should go into `tests/models/test_dagwarning.py` instead (matching the source tree’s structure).



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ephraimbuddy commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1061559655


##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,71 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models import DagModel
+from airflow.models.dagwarning import DagWarning
+from tests.test_utils.db import clear_db_dags
+
+
+class TestDagWarning:
+    def setup_method(self):
+        clear_db_dags()
+
+    def test_purge_inactive_dag_warnings(self, session):
+        """
+        Test that the purge_inactive_dag_warnings method deletes inactive dag warnings
+        """
+
+        dags = [DagModel(dag_id="dag_1", is_active=False), DagModel(dag_id="dag_2", is_active=True)]
+        for dag in dags:
+            session.add(dag)
+        session.commit()
+
+        dag_warnings = [
+            DagWarning("dag_1", "non-existent pool", "non-existent pool"),
+            DagWarning("dag_2", "non-existent pool", "non-existent pool"),
+        ]
+        for dag_warning in dag_warnings:
+            session.add(dag_warning)

Review Comment:
   ```suggestion
           session.add_all(dag_warnings)
   ```



##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,71 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models import DagModel
+from airflow.models.dagwarning import DagWarning
+from tests.test_utils.db import clear_db_dags
+
+
+class TestDagWarning:
+    def setup_method(self):
+        clear_db_dags()
+
+    def test_purge_inactive_dag_warnings(self, session):
+        """
+        Test that the purge_inactive_dag_warnings method deletes inactive dag warnings
+        """
+
+        dags = [DagModel(dag_id="dag_1", is_active=False), DagModel(dag_id="dag_2", is_active=True)]
+        for dag in dags:
+            session.add(dag)

Review Comment:
   ```suggestion
           session.add_all(dags)
   ```



##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,71 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models import DagModel
+from airflow.models.dagwarning import DagWarning
+from tests.test_utils.db import clear_db_dags
+
+
+class TestDagWarning:
+    def setup_method(self):
+        clear_db_dags()
+
+    def test_purge_inactive_dag_warnings(self, session):
+        """
+        Test that the purge_inactive_dag_warnings method deletes inactive dag warnings
+        """
+
+        dags = [DagModel(dag_id="dag_1", is_active=False), DagModel(dag_id="dag_2", is_active=True)]
+        for dag in dags:
+            session.add(dag)
+        session.commit()
+
+        dag_warnings = [
+            DagWarning("dag_1", "non-existent pool", "non-existent pool"),
+            DagWarning("dag_2", "non-existent pool", "non-existent pool"),
+        ]
+        for dag_warning in dag_warnings:
+            session.add(dag_warning)
+        session.commit()
+
+        DagWarning.purge_inactive_dag_warnings(session)
+
+        remaining_dag_warnings = session.query(DagWarning).all()
+        assert len(remaining_dag_warnings) == 1
+        assert remaining_dag_warnings[0].dag_id == "dag_2"
+
+    def test_retry_purge_inactive_dag_warnings(self):
+        """
+        Test that the purge_inactive_dag_warnings method calls the delete method twice
+        if the query throws an operationalError on the first call and works on the second attempt
+        """
+        self.session_mock = MagicMock()
+        self.delete_mock = MagicMock()
+        self.session_mock.query.return_value.filter.return_value.delete = self.delete_mock
+
+        self.delete_mock.side_effect = [OperationalError(None, None, "database timeout"), None]
+
+        DagWarning.purge_inactive_dag_warnings(self.session_mock)
+
+        # Assert that the delete method was called twice
+        assert self.delete_mock.call_count == 2

Review Comment:
   Why only twice? What's the default number of DB retries?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] michaelmicheal commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
michaelmicheal commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1061563650


##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,71 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models import DagModel
+from airflow.models.dagwarning import DagWarning
+from tests.test_utils.db import clear_db_dags
+
+
+class TestDagWarning:
+    def setup_method(self):
+        clear_db_dags()
+
+    def test_purge_inactive_dag_warnings(self, session):
+        """
+        Test that the purge_inactive_dag_warnings method deletes inactive dag warnings
+        """
+
+        dags = [DagModel(dag_id="dag_1", is_active=False), DagModel(dag_id="dag_2", is_active=True)]
+        for dag in dags:
+            session.add(dag)
+        session.commit()
+
+        dag_warnings = [
+            DagWarning("dag_1", "non-existent pool", "non-existent pool"),
+            DagWarning("dag_2", "non-existent pool", "non-existent pool"),
+        ]
+        for dag_warning in dag_warnings:
+            session.add(dag_warning)
+        session.commit()
+
+        DagWarning.purge_inactive_dag_warnings(session)
+
+        remaining_dag_warnings = session.query(DagWarning).all()
+        assert len(remaining_dag_warnings) == 1
+        assert remaining_dag_warnings[0].dag_id == "dag_2"
+
+    def test_retry_purge_inactive_dag_warnings(self):
+        """
+        Test that the purge_inactive_dag_warnings method calls the delete method twice
+        if the query throws an operationalError on the first call and works on the second attempt
+        """
+        self.session_mock = MagicMock()
+        self.delete_mock = MagicMock()
+        self.session_mock.query.return_value.filter.return_value.delete = self.delete_mock
+
+        self.delete_mock.side_effect = [OperationalError(None, None, "database timeout"), None]
+
+        DagWarning.purge_inactive_dag_warnings(self.session_mock)
+
+        # Assert that the delete method was called twice
+        assert self.delete_mock.call_count == 2

Review Comment:
   This tests that if it fails once and then works, we do in fact stop after the successful run. How much do I need to test that the retry does in fact work?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] michaelmicheal commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
michaelmicheal commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1059403394


##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,52 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+import unittest
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models.dagwarning import DagWarning
+
+
+class TestDagWarning(unittest.TestCase):

Review Comment:
   Are there any good examples of using the shared session fixture?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ephraimbuddy commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1059367759


##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,52 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+import unittest
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models.dagwarning import DagWarning
+
+
+class TestDagWarning(unittest.TestCase):

Review Comment:
   We now use pytest for our tests. With pytest, you have access to `session` fixture



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] michaelmicheal commented on a diff in pull request #28481: dagwarning, add error handling to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
michaelmicheal commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1055846581


##########
airflow/models/dagwarning.py:
##########
@@ -79,7 +83,12 @@ def purge_inactive_dag_warnings(cls, session: Session = NEW_SESSION) -> None:
             query = session.query(cls).filter(cls.dag_id.in_(dag_ids))
         else:
             query = session.query(cls).filter(cls.dag_id == DagModel.dag_id, DagModel.is_active == false())
-        query.delete(synchronize_session=False)
+        try:
+            query.delete(synchronize_session=False)
+        except OperationalError:
+            # If the purge query fails, it's not critical to the dag processor
+            # We ignore it so the dag processor manager doesn't exit
+            log.error("Failed to purge inactive dag_warnings, ignoring")

Review Comment:
   What do you think of this? Also, do you think we need to run this query every parsing loop? What if we only run it every x seconds and make that configurable?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] michaelmicheal commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
michaelmicheal commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1061545171


##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,52 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+import unittest
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models.dagwarning import DagWarning
+
+
+class TestDagWarning(unittest.TestCase):

Review Comment:
   @ephraimbuddy What do you think of the tests?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ephraimbuddy commented on a diff in pull request #28481: dagwarning, add error handling to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1058266047


##########
airflow/models/dagwarning.py:
##########
@@ -72,6 +73,11 @@ def purge_inactive_dag_warnings(cls, session: Session = NEW_SESSION) -> None:
 
         :return: None
         """
+        cls._purge_inactive_dag_warnings_with_retry(session)
+
+    @classmethod
+    @retry_db_transaction
+    def _purge_inactive_dag_warnings_with_retry(cls, session: Session) -> None:

Review Comment:
   LGTM, let's add a little test



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ephraimbuddy commented on a diff in pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1059675857


##########
tests/models/test_dagwarning.py:
##########
@@ -0,0 +1,52 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+import unittest
+from unittest.mock import MagicMock
+
+from sqlalchemy.exc import OperationalError
+
+from airflow.models.dagwarning import DagWarning
+
+
+class TestDagWarning(unittest.TestCase):

Review Comment:
   Check this: https://github.com/apache/airflow/blob/378c8faba19d21b29abfff1705a8e7624aeabdc4/tests/models/test_taskinstance.py#L507



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr merged pull request #28481: Add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
uranusjr merged PR #28481:
URL: https://github.com/apache/airflow/pull/28481


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] kaxil commented on a diff in pull request #28481: dagwarning, add error handling to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
kaxil commented on code in PR #28481:
URL: https://github.com/apache/airflow/pull/28481#discussion_r1054918242


##########
airflow/models/dagwarning.py:
##########
@@ -79,7 +83,12 @@ def purge_inactive_dag_warnings(cls, session: Session = NEW_SESSION) -> None:
             query = session.query(cls).filter(cls.dag_id.in_(dag_ids))
         else:
             query = session.query(cls).filter(cls.dag_id == DagModel.dag_id, DagModel.is_active == false())
-        query.delete(synchronize_session=False)
+        try:
+            query.delete(synchronize_session=False)
+        except OperationalError:
+            # If the purge query fails, it's not critical to the dag processor
+            # We ignore it so the dag processor manager doesn't exit
+            log.error("Failed to purge inactive dag_warnings, ignoring")

Review Comment:
   You can instead re-try the transaction https://github.com/apache/airflow/blob/23264fb820c179e9951ea9706f68b13a9b3fdbc0/airflow/utils/retries.py#L49



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] michaelmicheal commented on pull request #28481: dagwarning, add retry to purge_inactive_dag_warnings

Posted by GitBox <gi...@apache.org>.
michaelmicheal commented on PR #28481:
URL: https://github.com/apache/airflow/pull/28481#issuecomment-1371416934

   @uranusjr, @kaxil , is this good to merge?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] kaxil commented on pull request #28481: Add retry to purge_inactive_dag_warnings

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

   🎉 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org