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/08/21 23:38:29 UTC

[GitHub] [airflow] potiuk opened a new pull request, #25860: Improve logging of autput for Breeze commands in CI

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

   This change organizes better and improves overal output of the CI breeze
   commands, in the way that it makes it far more useful.
   
   1. No more separated output for running command and it's output in
      CI. They were displayed in separately foldable sections and it
      has been somewhat misleading. Now, the folded section contain both
      the command and its output.
   
   2. The heuristics to determine "short" name of a command has been
      improved and the foldable groups for the command are more similar
      to Github Actions runs (white color foldable) but also it has
      pink automatically rich-highlighted main command that is run.
   
   3. CI groups (foldable components) will no longer overlap. Github
      Actions does not support nested groups, so in case we start
      group in another group, the first group is "closed" automatically.
      With this change, if we start group inside of another group, the
      original group is not closed (but also the new group is not started
      and the nested command is not folded inside the main group.
   
   4. Progress output is much more consitent and nicer in case of
      parallel runs. We have it nicely wrapped in rich rules, showing
      how much time started with progress and single line of most
      recent interesting output from the parallel operation. We have
      "smarter" progress information implemented - showing much more
      useful one-liner truncated and caloured status from each of the
      running threads. Then all the individual outputs are nicer
      summarized in the folded groups after all of the parallel runs
      completed. No more spilling of the individual jobs into the
      progress monitoring.
   
   5. When running the same CI parallel commands locally, the output
      is adjusted to local terminal (no foldable groups and the output
      from parallel jobs is only printed if the job fails by default.
   
   This change has been extracted from #25678 as it grew out of it original
   scope.
   
   <!--
   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 an 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/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] potiuk commented on a diff in pull request #25860: Improve logging of output for Breeze commands in CI

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


##########
dev/breeze/src/airflow_breeze/utils/parallel.py:
##########
@@ -88,71 +83,142 @@ def get_last_lines_of_file(file_name: str, num_lines: int = 2) -> Tuple[List[str
     return last_lines, last_lines_no_colors
 
 
-DOCKER_BUILDX_PROGRESS_MATCHER = re.compile(r'\s*#(\d*) ')
-last_docker_build_lines: Dict[Output, str] = {}
+class ProgressLines(NamedTuple):
+    lines: List[str]
+    skip_truncation: List[bool]
+
+
+class AbstractProgressInfoMatcher(metaclass=ABCMeta):
+    @abstractmethod
+    def get_best_matching_lines(self, output: Output) -> Optional[ProgressLines]:
+        """
+        Return best matching lines of the output. It also indicates if the lines potentially need truncation
+        :param output: file that should be analysed for the output
+        :return: tuple of array of lines to print and boolean indications whether the lines need truncation
+        """
+
+
+class DockerBuildxProgressMatcher(AbstractProgressInfoMatcher):
+    DOCKER_BUILDX_PROGRESS_MATCHER = re.compile(r'\s*#(\d*) ')
+
+    def __init__(self):
+        self.last_docker_build_line: Optional[str] = None
+
+    def get_best_matching_lines(self, output) -> Optional[ProgressLines]:
+        last_lines, last_lines_no_colors = get_last_lines_of_file(output.file_name, num_lines=5)
+        best_progress: int = 0
+        best_line: Optional[str] = None
+        for index, line in enumerate(last_lines_no_colors):
+            match = DockerBuildxProgressMatcher.DOCKER_BUILDX_PROGRESS_MATCHER.match(line)
+            if match:
+                docker_progress = int(match.group(1))
+                if docker_progress > best_progress:
+                    best_progress = docker_progress
+                    best_line = last_lines[index]
+        if best_line is None:
+            best_line = self.last_docker_build_line
+        else:
+            self.last_docker_build_line = best_line
+        if best_line is None:
+            return None
+        return ProgressLines(lines=[best_line], skip_truncation=[False])
 
 
-def progress_method_docker_buildx(output: Output) -> Optional[str]:
-    last_lines, last_lines_no_colors = get_last_lines_of_file(output.file_name, num_lines=5)
-    best_progress: int = 0
-    best_line: Optional[str] = None
-    for index, line in enumerate(last_lines_no_colors):
-        match = DOCKER_BUILDX_PROGRESS_MATCHER.match(line)
-        if match:
-            docker_progress = int(match.group(1))
-            if docker_progress > best_progress:
-                best_progress = docker_progress
+class GenericRegexpProgressMatcher(AbstractProgressInfoMatcher):
+    def __init__(
+        self,
+        regexp: str,
+        lines_to_search: int,
+        regexp_for_joined_line: Optional[str] = None,
+        regexp_to_skip_truncation: Optional[str] = None,
+    ):
+        self.last_good_match: Optional[str] = None
+        self.matcher = re.compile(regexp)
+        self.lines_to_search = lines_to_search
+        self.matcher_for_joined_line = re.compile(regexp_for_joined_line) if regexp_for_joined_line else None

Review Comment:
   This one allows to see two last important lines of the output (for example in test output we will see):
   
   ![image](https://user-images.githubusercontent.com/595491/185924331-469d8d2a-f0b8-44c6-bef7-68feacfa3e69.png)
   



-- 
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] Bowrna commented on pull request #25860: Improve logging of output for Breeze commands in CI

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

   > * matching the "interesting" last line
   
   @potiuk Do you mean `interesting` by one that matches with regex?
   I think looking into code in `parallel.py` will give me better understanding of this part. Am i right in thinking that way?


-- 
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 commented on pull request #25860: Improve logging of autput for Breeze commands in CI

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

   Right. I have run it with "full tests needed and you can see the result of in "build image" "pull image". in #25678  I am useing it in Helm Tests and @bowrna in #23715 might use it after we merge this one.


-- 
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 commented on pull request #25860: Improve logging of output for Breeze commands in CI

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

   > I am checking the changes and using it in breeze test part @potiuk
   
   Cool. It is a bit more involved - with decorators and such, but It is super versatile. You might also take a look how I've implemented the regexps for Helm Tests in https://github.com/apache/airflow/pull/25678/files#diff-ec0edb35ebfa2f51b6030b54f72eb90b0661eb310b0baea6e9c605f49c15123dR1059 
   
   There are two regexps for tests:
   1) matching the "interesting" last line
   2) matching the "optionall preceding line"
   
   This is the way how in the output of tests you can see two lines of the test progress (which is crucial to see the percentag e progress of tests) while the tests are running


-- 
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] Bowrna commented on pull request #25860: Improve logging of output for Breeze commands in CI

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

   > cc: @Bowrna - once we merge it, you will be able to re-use what I added here without waiting for #25678 - I am struggling there a bit with stability of cluster creations, so it might take a bit more time to complete. But this one has a much nicer and much more reusable way of running parallel tests and presenting the output.
   
   I am checking the changes and using it in breeze test part @potiuk 


-- 
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 commented on pull request #25860: Improve logging of autput for Breeze commands in CI

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

   cc: @Bowrna - once we merge it, you will be able to re-use what I added here without waiting for #25678 - I am struggling there a bit with stability of cluster creations, so it might take a bit more time to complete. But this one has a much nicer and much more reusable way of running parallel tests and presenting the output.


-- 
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 commented on pull request #25860: Improve logging of output for Breeze commands in CI

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

   In K8S tests this is achieved by 
   
   ```
   K8S_TEST_PROGRESS_REGEXP = r'.*airflow-python-[0-9.]+-v[0-9.].*|^kubernetes_tests/.*'
   PERCENT_K8S_TEST_PROGRESS_REGEXP = r'^kubernetes_tests/.*\[[ 0-9%]*\].*'
   K8S_SKIP_TRUNCATION_REGEXP = r'^kubernetes_tests/.*'
   
       with ci_group(f"Running tests for: {combos}"):
           with run_with_pool(
               parallelism=parallelism,
               all_params=combo_titles,
               progress_matcher=GenericRegexpProgressMatcher(
                   K8S_TEST_PROGRESS_REGEXP,
                   lines_to_search=15,
                   regexp_for_joined_line=PERCENT_K8S_TEST_PROGRESS_REGEXP,
                   regexp_to_skip_truncation=K8S_SKIP_TRUNCATION_REGEXP,
               ),
           ) as (pool, outputs):
   ```


-- 
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 commented on pull request #25860: Improve logging of output for Breeze commands in CI

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

   > @potiuk Do you mean interesting by one that matches with regex?
   
   Correct. The thing is that in many cases we are interested to see only some output and it might be intertwined with not-interesting ones. So regexp matches onle those lines that provide some info about the progress..
   
   example here:
   
   ```
   
     Database has been reset
     
     
     Those integrations are disabled:  kerberos mongo redis cassandra openldap trino pinot rabbitmq
     
     
     Fixing ownership of 4147 root owned files on Linux
     
     Fixed ownership of generated files.
     
     
     Running tests tests/core tests/executors tests/jobs tests/models tests/serialization tests/ti_deps tests/utils
     
     
     Starting the tests with those pytest arguments: --verbosity=0 --strict-markers --durations=100 --maxfail=50 --color=yes --junitxml=/files/test_result-Core-postgres.xml --timeouts-order moi --setup-timeout=60 --execution-timeout=60 --teardown-timeout=60 -rfEX --with-db-init --cov=airflow/ --cov-config=.coveragerc --cov-report=xml:/files/coverage-Core-postgres.xml tests/core tests/executors tests/jobs tests/models tests/serialization tests/ti_deps tests/utils
     
     ============================= test session starts ==============================
     platform linux -- Python 3.10.6, pytest-6.2.5, py-1.11.0, pluggy-1.0.0
     rootdir: /opt/airflow, configfile: pytest.ini
     plugins: requests-mock-1.9.3, xdist-2.5.0, timeouts-1.2.1, rerunfailures-9.1.1, instafail-0.4.2, httpx-0.21.0, forked-1.4.0, cov-3.0.0, asyncio-0.19.0, flaky-3.7.0, anyio-3.6.1
     asyncio: mode=strict
     setup timeout: 60.0s, execution timeout: 60.0s, teardown timeout: 60.0s
     collected 1978 items
     
     tests/core/test_config_templates.py ....                                 [  0%]
     tests/core/test_configuration.py ....................................... [  2%]
     ...............................................                          [  4%]
     tests/core/test_core.py ......                                           [  4%]
     tests/core/test_example_dags_system.py sssss                             [  5%]
     tests/core/test_impersonation_tests.py sssss                             [  5%]
     tests/core/test_logging_config.py ...........                            [  5%]
     tests/core/test_providers_manager.py .....................               [  6%]
     tests/core/test_sentry.py ....                                           [  7%]
     tests/core/test_settings.py ..........                                   [  7%]
     tests/core/test_sqlalchemy_config.py ...                                 [  7%]
     tests/core/test_stats.py ....................................            [  9%]
     tests/core/test_templates.py .......                                     [ 10%]
     tests/executors/test_base_executor.py .............                      [ 10%]
     tests/executors/test_celery_executor.py sssss...........ssss             [ 11%]
     tests/executors/test_celery_kubernetes_executor.py ....................  [ 12%]
     tests/executors/test_dask_executor.py ..ss....                           [ 13%]
     tests/executors/test_debug_executor.py ......                            [ 13%]
     tests/executors/test_executor_loader.py .......                          [ 13%]
     tests/executors/test_kubernetes_executor.py ............................ [ 15%]
     ...............                                                          [ 15%]
     tests/executors/test_local_executor.py .....                             [ 16%]
     tests/executors/test_local_kubernetes_executor.py .....                  [ 16%]
     tests/executors/test_sequential_executor.py .                            [ 16%]
     tests/jobs/test_backfill_job.py ..x..................................... [ 18%]
     ....ssssss..                                                             [ 19%]
     tests/jobs/test_base_job.py ........                                     [ 19%]
     tests/jobs/test_local_task_job.py .......s...........s.                  [ 20%]
     tests/jobs/test_scheduler_job.py ...........F........................... [ 22%]
     ...........................................s............................ [ 26%]
     ..............ss.................................                     
   ```
   
   we are not interested in any of the preparation start, but what we want to show as current progress are literally the two last lines:
   
   ```
     ...........................................s............................ [ 26%]
     ..............ss.................................                     
   ```
   
   Ideally with colour (this example is taken from https://github.com/apache/airflow/runs/7977534034?check_suite_focus=true) and you can see that the % is red indicating that we had some failure before.
   
   The Progress Monitor already handles that. you just need to specify the right regexps:
   
   * matching all the "interesting lines" - in our case those with progress
   * matching the previous interesing line (they will be mostly the same but there are some cases in Helm chart where I wanted to seperate them because I am mixing "cluster creation" "cluster deployment' and "test execution" in single output so  sometimes I have one-line output, sometimes 2 lines.
   * we need to avoid truncation for those "test" lines - and the regexp for that  in case of regular tests will be the same
   
   So I think the "unit test" will be simpe - just one regexp matching the line with ....  should do. It's a bit tricky though - because the progress line might be quite different - mostly it has "." but sometimes "s, x, f, F ' and few others. So you need to make some smart regexp there to match all of them but not match "uninteresting" lines. You can take the output of some of our tests in CI to test those matches.
   
   > I think looking into code in parallel.py will give me better understanding of this part. Am i right in thinking that way?
   Correct. The Parallel implemetnation is the right way to look at


-- 
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 #25860: Improve logging of output for Breeze commands in CI

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


-- 
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 commented on pull request #25860: Improve logging of autput for Breeze commands in CI

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

   This one is extracted from #25678 and it provides a number of improvements in the UI of CI output (see the description). Some screenshots:
   
   
   Parallel runs with progress:
   
   <img width="1518" alt="Screenshot 2022-08-22 at 01 54 01" src="https://user-images.githubusercontent.com/595491/185816646-3d33f5e2-4e2e-4539-849f-6d7fc97e64c5.png">
   
   Series of commands with folded output and command defaults:
   
   <img width="748" alt="Screenshot 2022-08-22 at 01 57 36" src="https://user-images.githubusercontent.com/595491/185816704-8745bd27-ac0d-48b5-9f6e-2d53e5dee613.png">
   
   Some command unfolded:
   
   <img width="907" alt="Screenshot 2022-08-22 at 01 58 12" src="https://user-images.githubusercontent.com/595491/185816738-bcc0f69f-5b26-477c-b5ce-d8a1c4057a6c.png">
   
   Another example:
   
   <img width="770" alt="Screenshot 2022-08-22 at 02 01 07" src="https://user-images.githubusercontent.com/595491/185816876-3460a1e4-86a9-49bd-b3cb-710e2cccac73.png">
   
   
   
   
   


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