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/15 09:41:53 UTC

[GitHub] [airflow] Taragolis opened a new pull request, #28378: Migrate remaining Core tests to `pytest`

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

   Migrate remaining Core tests to `pytest` in selected directories:
   - `tests/core/*`
   - `tests/executors/*`
   - `tests/jobs/*`
   - `tests/models/*`
   - `tests/serialization/*`
   - `tests/ti_deps/*`
   - `tests/utils/*`


-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/models/test_serialized_dag.py:
##########
@@ -39,33 +39,21 @@ def make_example_dags(module):
     return dagbag.dags
 
 
-def clear_db_serialized_dags():
-    with create_session() as session:
-        session.query(SDM).delete()
-
-
-@parameterized_class(
-    [
-        {"compress_serialized_dags": "False"},
-        {"compress_serialized_dags": "True"},
-    ]
-)

Review Comment:
   This parameters always evaluate as `True`, so tests alway runs against compressed compress serialized DAGs.
   
   https://github.com/apache/airflow/blob/2794662459b6d2f40b55b9b1d39eed14b5c3c5e9/airflow/models/serialized_dag.py#L107-L112
   
   This fixed



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/utils/test_process_utils.py:
##########
@@ -202,15 +202,16 @@ def test_should_restore_state_when_exception(self):
             assert "TEST_NOT_EXISTS" not in os.environ
 
 
-class TestCheckIfPidfileProcessIsRunning(unittest.TestCase):
+class TestCheckIfPidfileProcessIsRunning:
     def test_ok_if_no_file(self):
         check_if_pidfile_process_is_running("some/pid/file", process_name="test")
 
     def test_remove_if_no_process(self):
         # Assert file is deleted
         with pytest.raises(FileNotFoundError):
             with NamedTemporaryFile("+w") as f:
-                f.write("19191919191919191991")
+                # limit pid as max of int32, otherwise this test could fail on some platform
+                f.write(f"{2**31 - 1}")

Review Comment:
   Small fix for success execute test in macOS ARM, no idea about behaviour in Intel based mac
   
   ```console
   self = <psutil._psosx.Process object at 0x105e7cd40>
   
       @wrap_exceptions
       @memoize_when_activated
       def _get_kinfo_proc(self):
           # Note: should work with all PIDs without permission issues.
   >       ret = cext.proc_kinfo_oneshot(self.pid)
   E       OverflowError: Python int too large to convert to C long
   ```



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/core/test_impersonation_tests.py:
##########
@@ -161,14 +144,15 @@ def test_no_impersonation(self):
             "test_superuser",
         )
 
-    @unittest.mock.patch.dict("os.environ", AIRFLOW__CORE__DEFAULT_IMPERSONATION=TEST_USER)
+    @mock.patch.dict("os.environ", AIRFLOW__CORE__DEFAULT_IMPERSONATION=TEST_USER)
     def test_default_impersonation(self):
         """
         If default_impersonation=TEST_USER, tests that the job defaults
         to running as TEST_USER for a test without run_as_user set
         """
         self.run_backfill("test_default_impersonation", "test_deelevated_user")
 
+    @pytest.mark.execution_timeout(150)

Review Comment:
   `TestImpersonation::test_impersonation_subdag` test almost hit timeout (60s) in the CI
   
   ```console
   ============================ slowest 100 durations =============================
     48.91s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation_subdag
     24.39s call     tests/core/test_impersonation_tests.py::TestImpersonationWithCustomPythonPath::test_impersonation_custom
     12.27s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation
     12.18s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_default_impersonation
     6.40s setup    tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation
     6.34s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_no_impersonation
   ```



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/models/test_serialized_dag.py:
##########
@@ -39,33 +39,21 @@ def make_example_dags(module):
     return dagbag.dags
 
 
-def clear_db_serialized_dags():
-    with create_session() as session:
-        session.query(SDM).delete()
-
-
-@parameterized_class(
-    [
-        {"compress_serialized_dags": "False"},
-        {"compress_serialized_dags": "True"},
-    ]
-)

Review Comment:
   This parameters always evaluate as `True`, so tests runs against compressed serialized DAGs.
   
   https://github.com/apache/airflow/blob/2794662459b6d2f40b55b9b1d39eed14b5c3c5e9/airflow/models/serialized_dag.py#L107-L112
   
   fixed



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/core/test_impersonation_tests.py:
##########
@@ -161,14 +144,15 @@ def test_no_impersonation(self):
             "test_superuser",
         )
 
-    @unittest.mock.patch.dict("os.environ", AIRFLOW__CORE__DEFAULT_IMPERSONATION=TEST_USER)
+    @mock.patch.dict("os.environ", AIRFLOW__CORE__DEFAULT_IMPERSONATION=TEST_USER)
     def test_default_impersonation(self):
         """
         If default_impersonation=TEST_USER, tests that the job defaults
         to running as TEST_USER for a test without run_as_user set
         """
         self.run_backfill("test_default_impersonation", "test_deelevated_user")
 
+    @pytest.mark.execution_timeout(150)

Review Comment:
   `TestImpersonation::test_impersonation_subdag` test almost reach timeout (60s) in the CI
   
   ```console
   ============================ slowest 100 durations =============================
     48.91s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation_subdag
     24.39s call     tests/core/test_impersonation_tests.py::TestImpersonationWithCustomPythonPath::test_impersonation_custom
     12.27s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation
     12.18s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_default_impersonation
     6.40s setup    tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation
     6.34s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_no_impersonation
   ```



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/models/test_serialized_dag.py:
##########
@@ -39,33 +39,21 @@ def make_example_dags(module):
     return dagbag.dags
 
 
-def clear_db_serialized_dags():
-    with create_session() as session:
-        session.query(SDM).delete()
-
-
-@parameterized_class(
-    [
-        {"compress_serialized_dags": "False"},
-        {"compress_serialized_dags": "True"},
-    ]
-)

Review Comment:
   This parameters always evaluate as `True`, so tests runs against compressed compress serialized DAGs.
   
   https://github.com/apache/airflow/blob/2794662459b6d2f40b55b9b1d39eed14b5c3c5e9/airflow/models/serialized_dag.py#L107-L112
   
   fixed



-- 
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] potiuk merged pull request #28378: Migrate remaining Core tests to `pytest`

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


-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/utils/test_process_utils.py:
##########
@@ -202,15 +202,16 @@ def test_should_restore_state_when_exception(self):
             assert "TEST_NOT_EXISTS" not in os.environ
 
 
-class TestCheckIfPidfileProcessIsRunning(unittest.TestCase):
+class TestCheckIfPidfileProcessIsRunning:
     def test_ok_if_no_file(self):
         check_if_pidfile_process_is_running("some/pid/file", process_name="test")
 
     def test_remove_if_no_process(self):
         # Assert file is deleted
         with pytest.raises(FileNotFoundError):
             with NamedTemporaryFile("+w") as f:
-                f.write("19191919191919191991")
+                # limit pid as max of int32, otherwise this test could fail on some platform
+                f.write(f"{2**31 - 1}")

Review Comment:
   Small fix for execute test in macOS ARM, no idea about behaviour in Intel based mac
   
   ```console
   self = <psutil._psosx.Process object at 0x105e7cd40>
   
       @wrap_exceptions
       @memoize_when_activated
       def _get_kinfo_proc(self):
           # Note: should work with all PIDs without permission issues.
   >       ret = cext.proc_kinfo_oneshot(self.pid)
   E       OverflowError: Python int too large to convert to C long
   ```



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/core/test_impersonation_tests.py:
##########
@@ -114,23 +92,28 @@ def create_user():
 
 
 @pytest.mark.quarantined
-class TestImpersonation(unittest.TestCase):
+class TestImpersonation:

Review Comment:
   `TestImpersonation::test_impersonation_subdag` test almost reach timeout (60s)
   
   ```console
   ============================ slowest 100 durations =============================
     48.91s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation_subdag
     24.39s call     tests/core/test_impersonation_tests.py::TestImpersonationWithCustomPythonPath::test_impersonation_custom
     12.27s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation
     12.18s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_default_impersonation
     6.40s setup    tests/core/test_impersonation_tests.py::TestImpersonation::test_impersonation
     6.34s call     tests/core/test_impersonation_tests.py::TestImpersonation::test_no_impersonation
   ```



-- 
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] Taragolis commented on a diff in pull request #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/executors/test_celery_executor.py:
##########
@@ -100,15 +102,14 @@ def teardown_method(self) -> None:
 
     @pytest.mark.quarantined
     @pytest.mark.backend("mysql", "postgres")
-    def test_exception_propagation(self):
-
-        with _prepare_app(), self.assertLogs(celery_executor.log) as cm:
+    def test_exception_propagation(self, caplog):

Review Comment:
   This test skipped in Quarantined test in CI because it run in SQLite backend



-- 
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 #28378: Migrate remaining Core tests to `pytest`

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


##########
tests/core/test_impersonation_tests.py:
##########
@@ -47,55 +46,33 @@
 logger = logging.getLogger(__name__)
 
 
-def mock_custom_module_path(path: str):
-    """
-    This decorator adds a path to sys.path to simulate running the current script with
-    the :envvar:`PYTHONPATH` environment variable set and sets the environment variable
-    :envvar:`PYTHONPATH` to change the module load directory for child scripts.
-    """
-
-    def wrapper(func):
-        @functools.wraps(func)
-        def decorator(*args, **kwargs):
-            copy_sys_path = deepcopy(sys.path)
-            sys.path.append(path)
-            try:
-                with unittest.mock.patch.dict("os.environ", {"PYTHONPATH": path}):
-                    return func(*args, **kwargs)
-            finally:
-                sys.path = copy_sys_path
-
-        return decorator
-
-    return wrapper
+@pytest.fixture
+def check_original_docker_image():
+    if not os.path.isfile("/.dockerenv") or os.environ.get("PYTHON_BASE_IMAGE") is None:
+        raise pytest.skip(
+            "Adding/removing a user as part of a test is very bad for host os "
+            "(especially if the user already existed to begin with on the OS), "
+            "therefore we check if we run inside a the official docker container "
+            "and only allow to run the test there. This is done by checking /.dockerenv file "
+            "(always present inside container) and checking for PYTHON_BASE_IMAGE variable."
+        )
+    yield

Review Comment:
   ```suggestion
   ```
   
   Not needed



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