mik-laj commented on a change in pull request #6627: [AIRFLOW-5931] Use os.fork 
when appropriate to speed up task execution.
URL: https://github.com/apache/airflow/pull/6627#discussion_r351015019
 
 

 ##########
 File path: airflow/task/task_runner/standard_task_runner.py
 ##########
 @@ -17,28 +17,96 @@
 # specific language governing permissions and limitations
 # under the License.
 
+import os
+
 import psutil
+from setproctitle import setproctitle
 
 from airflow.task.task_runner.base_task_runner import BaseTaskRunner
 from airflow.utils.helpers import reap_process_group
 
+CAN_FORK = hasattr(os, 'fork')
+
 
 class StandardTaskRunner(BaseTaskRunner):
     """
     Runs the raw Airflow task by invoking through the Bash shell.
     """
     def __init__(self, local_task_job):
         super().__init__(local_task_job)
+        self._rc = None
 
     def start(self):
-        self.process = self.run_command()
+        if CAN_FORK and not self.run_as_user:
+            self.process = self._start_by_fork()
+        else:
+            self.process = self._start_by_exec()
+
+    def _start_by_exec(self):
+        subprocess = self.run_command()
+        return psutil.Process(subprocess.pid)
+
+    def _start_by_fork(self):
+        pid = os.fork()
+        if pid:
+            self.log.info("Started process %d to run task", pid)
+            return psutil.Process(pid)
+        else:
+            from airflow.bin.cli import get_parser
+            from airflow.logging_config import configure_logging
+            import signal
+            import airflow.settings as settings
+
+            signal.signal(signal.SIGINT, signal.SIG_DFL)
+            signal.signal(signal.SIGTERM, signal.SIG_DFL)
+            # Start a new process group
+            os.setpgid(0, 0)
+
+            configure_logging()
+
+            # Force a new SQLAlchemy session. We can't share open DB handles
+            # between process. The cli code will re-create this as part of its
+            # normal startup
+            settings.engine.pool.dispose()
+            settings.engine.dispose()
 
-    def return_code(self):
-        return self.process.poll()
+            parser = get_parser()
+            args = parser.parse_args(self._command[1:])
 
 Review comment:
   ```suggestion
               # [1:] - remove "airflow" from the command
               args = parser.parse_args(self._command[1:])
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to