You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "Bisk1 (via GitHub)" <gi...@apache.org> on 2023/10/14 17:22:45 UTC

[PR] Refactor commands to unify daemon context handling [airflow]

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

   This refactoring is a follow-up of https://github.com/apache/airflow/pull/34931
   - unified approach to daemon process creation and moved all daemon-related code to shared util daemon_utils.py.
   - moved async thread creation for scheduler so that it happens after entering daemon context (will prevent possible bugs like https://github.com/apache/airflow/issues/34816#event-10654536327 for scheduler)
   - reused some utils like removing PID file of a dead process for all other commands in daemon mode
   - one change in PID file name generation for consistency: the PID file for the main thread of webserver process (and internal-api) is now airflow-webserver.pid (previosuly: airflow-webserver-monitor.pid) and the PID file for gunicorn master thread is airflow-webserver-gunicorn-master.pid (previously: airflow-webserver.pid); same for internal-api
   
   
   <!--
    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.
    -->
   
   <!--
   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/
   -->
   
   
   
   <!-- Please keep an empty line above the dashes. -->
   ---
   **^ 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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1367723811


##########
airflow/cli/commands/daemon_utils.py:
##########
@@ -0,0 +1,89 @@
+# 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 signal
+from argparse import Namespace
+from typing import Callable
+
+from daemon import daemon
+from daemon.pidfile import TimeoutPIDLockFile
+
+from airflow import settings
+from airflow.utils.cli import setup_locations, setup_logging, sigint_handler, sigquit_handler
+from airflow.utils.process_utils import check_if_pidfile_process_is_running
+
+
+def run_command_with_daemon_mode(
+    args: Namespace,
+    process_name: str,
+    callback: Callable,
+    should_setup_logging: bool = False,
+    umask: str = settings.DAEMON_UMASK,
+    pid_file: str | None = None,
+):
+    """Run the command in daemon process if enabled or in this process if not.
+
+    Parameters
+    ----------
+    args : Namespace
+        The set of arguments passed to the original CLI command
+    process_name : str
+        Process name used in naming log and PID files for the daemon
+    callback : Callable
+        The actual command to run with or without daemon context
+    should_setup_logging : bool
+        If true, then a log file handler for the daemon process will be created
+    umask : str
+        File access creation mask ("umask") to set for the process on daemon start
+    pid_file : str
+        If specified, this file path us used to store daemon process PID.
+        If not specified, a file path is generated with the default pattern.

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.

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

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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #34945:
URL: https://github.com/apache/airflow/pull/34945#issuecomment-1776076860

   @Taragolis ?


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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "daniel-dylag-openx (via GitHub)" <gi...@apache.org>.
daniel-dylag-openx commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1369137222


##########
airflow/cli/commands/celery_command.py:
##########
@@ -214,33 +187,31 @@ def worker(args):
         # executed.
         maybe_patch_concurrency(["-P", pool])
 
-    if args.daemon:
-        # Run Celery worker as daemon
-        handle = setup_logging(log_file)
-
-        with open(stdout, "a") as stdout_handle, open(stderr, "a") as stderr_handle:
-            if args.umask:
-                umask = args.umask
-            else:
-                umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
-
-            stdout_handle.truncate(0)
-            stderr_handle.truncate(0)
-
-            daemon_context = daemon.DaemonContext(
-                files_preserve=[handle],
-                umask=int(umask, 8),
-                stdout=stdout_handle,
-                stderr=stderr_handle,
-            )
-            with daemon_context, _serve_logs(skip_serve_logs):
-                celery_app.worker_main(options)
+    _, stdout, stderr, log_file = setup_locations(
+        process=WORKER_PROCESS_NAME,
+        stdout=args.stdout,
+        stderr=args.stderr,
+        log=args.log_file,
+    )
 
-    else:
-        # Run Celery worker in the same process
+    def run_celery_worker():
         with _serve_logs(skip_serve_logs):
             celery_app.worker_main(options)
 
+    if args.umask:
+        umask = args.umask
+    else:
+        umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
+
+    run_command_with_daemon_mode(
+        args,

Review Comment:
   Changed all parametry to keyword-only



##########
airflow/cli/commands/celery_command.py:
##########
@@ -214,33 +187,31 @@ def worker(args):
         # executed.
         maybe_patch_concurrency(["-P", pool])
 
-    if args.daemon:
-        # Run Celery worker as daemon
-        handle = setup_logging(log_file)
-
-        with open(stdout, "a") as stdout_handle, open(stderr, "a") as stderr_handle:
-            if args.umask:
-                umask = args.umask
-            else:
-                umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
-
-            stdout_handle.truncate(0)
-            stderr_handle.truncate(0)
-
-            daemon_context = daemon.DaemonContext(
-                files_preserve=[handle],
-                umask=int(umask, 8),
-                stdout=stdout_handle,
-                stderr=stderr_handle,
-            )
-            with daemon_context, _serve_logs(skip_serve_logs):
-                celery_app.worker_main(options)
+    _, stdout, stderr, log_file = setup_locations(
+        process=WORKER_PROCESS_NAME,
+        stdout=args.stdout,
+        stderr=args.stderr,
+        log=args.log_file,
+    )
 
-    else:
-        # Run Celery worker in the same process
+    def run_celery_worker():
         with _serve_logs(skip_serve_logs):
             celery_app.worker_main(options)
 
+    if args.umask:
+        umask = args.umask
+    else:
+        umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
+
+    run_command_with_daemon_mode(
+        args,

Review Comment:
   Changed all parametry to keyword-only



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on PR #34945:
URL: https://github.com/apache/airflow/pull/34945#issuecomment-1763057616

   Tested those commands manually and looks good. Unfortunately standard Python test frameworks can't handle forking processes well so no automated 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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1369154410


##########
airflow/cli/commands/internal_api_command.py:
##########
@@ -148,60 +139,54 @@ def kill_proc(signum, _):
                 gunicorn_master_proc.kill()
             sys.exit(0)
 
-        def monitor_gunicorn(gunicorn_master_pid: int):
+        def monitor_gunicorn(gunicorn_master_proc: psutil.Process | subprocess.Popen):

Review Comment:
   add typing; turns out there was some error in code that I fixed by copying&pasting the same section from webserver_command.py (BTW these 2 files have a lot of duplication so could use more refactoring)



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1369152054


##########
airflow/cli/commands/celery_command.py:
##########
@@ -68,28 +67,7 @@ def flower(args):
     if args.flower_conf:
         options.append(f"--conf={args.flower_conf}")
 
-    if args.daemon:
-        pidfile, stdout, stderr, _ = setup_locations(
-            process="flower",
-            pid=args.pid,
-            stdout=args.stdout,
-            stderr=args.stderr,
-            log=args.log_file,
-        )
-        with open(stdout, "a") as stdout, open(stderr, "a") as stderr:
-            stdout.truncate(0)
-            stderr.truncate(0)
-
-            ctx = daemon.DaemonContext(
-                pidfile=TimeoutPIDLockFile(pidfile, -1),
-                stdout=stdout,
-                stderr=stderr,
-                umask=int(settings.DAEMON_UMASK, 8),
-            )
-            with ctx:
-                celery_app.start(options)
-    else:
-        celery_app.start(options)
+    run_command_with_daemon_mode(args, "flower", lambda: celery_app.start(options))

Review Comment:
   sure, renamed



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1368962020


##########
airflow/cli/commands/internal_api_command.py:
##########
@@ -148,60 +139,54 @@ def kill_proc(signum, _):
                 gunicorn_master_proc.kill()
             sys.exit(0)
 
-        def monitor_gunicorn(gunicorn_master_pid: int):
+        def monitor_gunicorn(gunicorn_master_proc: psutil.Process | subprocess.Popen):

Review Comment:
   I remember this psutikk/subprocess dualism hit us before. Nice to have it typed here, bit it's not typed in kill_proc - I do not know by heart if all teh methods are supported in both ? Can you add type hint there please?



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1369152541


##########
airflow/cli/commands/celery_command.py:
##########
@@ -214,33 +187,31 @@ def worker(args):
         # executed.
         maybe_patch_concurrency(["-P", pool])
 
-    if args.daemon:
-        # Run Celery worker as daemon
-        handle = setup_logging(log_file)
-
-        with open(stdout, "a") as stdout_handle, open(stderr, "a") as stderr_handle:
-            if args.umask:
-                umask = args.umask
-            else:
-                umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
-
-            stdout_handle.truncate(0)
-            stderr_handle.truncate(0)
-
-            daemon_context = daemon.DaemonContext(
-                files_preserve=[handle],
-                umask=int(umask, 8),
-                stdout=stdout_handle,
-                stderr=stderr_handle,
-            )
-            with daemon_context, _serve_logs(skip_serve_logs):
-                celery_app.worker_main(options)
+    _, stdout, stderr, log_file = setup_locations(
+        process=WORKER_PROCESS_NAME,
+        stdout=args.stdout,
+        stderr=args.stderr,
+        log=args.log_file,
+    )
 
-    else:
-        # Run Celery worker in the same process
+    def run_celery_worker():
         with _serve_logs(skip_serve_logs):
             celery_app.worker_main(options)
 
+    if args.umask:
+        umask = args.umask
+    else:
+        umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
+
+    run_command_with_daemon_mode(
+        args,

Review Comment:
   changed all params to keyword-only



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1369154410


##########
airflow/cli/commands/internal_api_command.py:
##########
@@ -148,60 +139,54 @@ def kill_proc(signum, _):
                 gunicorn_master_proc.kill()
             sys.exit(0)
 
-        def monitor_gunicorn(gunicorn_master_pid: int):
+        def monitor_gunicorn(gunicorn_master_proc: psutil.Process | subprocess.Popen):

Review Comment:
   added types and looked closer at kill_proc, turns out there was some error in code related to this duality that I fixed by copying&pasting the same section from webserver_command.py (BTW these 2 files have a lot of duplication so could use more refactoring)



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1367714633


##########
airflow/cli/commands/daemon_utils.py:
##########
@@ -0,0 +1,89 @@
+# 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 signal
+from argparse import Namespace
+from typing import Callable
+
+from daemon import daemon
+from daemon.pidfile import TimeoutPIDLockFile
+
+from airflow import settings
+from airflow.utils.cli import setup_locations, setup_logging, sigint_handler, sigquit_handler
+from airflow.utils.process_utils import check_if_pidfile_process_is_running
+
+
+def run_command_with_daemon_mode(
+    args: Namespace,
+    process_name: str,
+    callback: Callable,
+    should_setup_logging: bool = False,
+    umask: str = settings.DAEMON_UMASK,
+    pid_file: str | None = None,
+):
+    """Run the command in daemon process if enabled or in this process if not.
+
+    Parameters
+    ----------
+    args : Namespace
+        The set of arguments passed to the original CLI command
+    process_name : str
+        Process name used in naming log and PID files for the daemon
+    callback : Callable
+        The actual command to run with or without daemon context
+    should_setup_logging : bool
+        If true, then a log file handler for the daemon process will be created
+    umask : str
+        File access creation mask ("umask") to set for the process on daemon start
+    pid_file : str
+        If specified, this file path us used to store daemon process PID.
+        If not specified, a file path is generated with the default pattern.

Review Comment:
   This should be in reStructuredText docstring style



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1368938322


##########
airflow/cli/commands/celery_command.py:
##########
@@ -214,33 +187,31 @@ def worker(args):
         # executed.
         maybe_patch_concurrency(["-P", pool])
 
-    if args.daemon:
-        # Run Celery worker as daemon
-        handle = setup_logging(log_file)
-
-        with open(stdout, "a") as stdout_handle, open(stderr, "a") as stderr_handle:
-            if args.umask:
-                umask = args.umask
-            else:
-                umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
-
-            stdout_handle.truncate(0)
-            stderr_handle.truncate(0)
-
-            daemon_context = daemon.DaemonContext(
-                files_preserve=[handle],
-                umask=int(umask, 8),
-                stdout=stdout_handle,
-                stderr=stderr_handle,
-            )
-            with daemon_context, _serve_logs(skip_serve_logs):
-                celery_app.worker_main(options)
+    _, stdout, stderr, log_file = setup_locations(
+        process=WORKER_PROCESS_NAME,
+        stdout=args.stdout,
+        stderr=args.stderr,
+        log=args.log_file,
+    )
 
-    else:
-        # Run Celery worker in the same process
+    def run_celery_worker():
         with _serve_logs(skip_serve_logs):
             celery_app.worker_main(options)
 
+    if args.umask:
+        umask = args.umask
+    else:
+        umask = conf.get("celery", "worker_umask", fallback=settings.DAEMON_UMASK)
+
+    run_command_with_daemon_mode(
+        args,

Review Comment:
   Maybe we should use keywords-only params here? 



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #34945:
URL: https://github.com/apache/airflow/pull/34945#discussion_r1368935064


##########
airflow/cli/commands/celery_command.py:
##########
@@ -68,28 +67,7 @@ def flower(args):
     if args.flower_conf:
         options.append(f"--conf={args.flower_conf}")
 
-    if args.daemon:
-        pidfile, stdout, stderr, _ = setup_locations(
-            process="flower",
-            pid=args.pid,
-            stdout=args.stdout,
-            stderr=args.stderr,
-            log=args.log_file,
-        )
-        with open(stdout, "a") as stdout, open(stderr, "a") as stderr:
-            stdout.truncate(0)
-            stderr.truncate(0)
-
-            ctx = daemon.DaemonContext(
-                pidfile=TimeoutPIDLockFile(pidfile, -1),
-                stdout=stdout,
-                stderr=stderr,
-                umask=int(settings.DAEMON_UMASK, 8),
-            )
-            with ctx:
-                celery_app.start(options)
-    else:
-        celery_app.start(options)
+    run_command_with_daemon_mode(args, "flower", lambda: celery_app.start(options))

Review Comment:
   ```suggestion
       run_command_with_daemon_option(args, "flower", lambda: celery_app.start(options))
   ```
   
   Maybe? The name suggest that it's always daemon but it is based on args.daemon .



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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "Bisk1 (via GitHub)" <gi...@apache.org>.
Bisk1 commented on PR #34945:
URL: https://github.com/apache/airflow/pull/34945#issuecomment-1763474577

   converting to draft becuase I still need to fix some unit 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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk merged PR #34945:
URL: https://github.com/apache/airflow/pull/34945


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


Re: [PR] Refactor commands to unify daemon context handling [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #34945:
URL: https://github.com/apache/airflow/pull/34945#issuecomment-1776738164

   Nice one @Bisk1 !


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