TobKed commented on a change in pull request #12814:
URL: https://github.com/apache/airflow/pull/12814#discussion_r565356744



##########
File path: airflow/providers/apache/beam/hooks/beam.py
##########
@@ -0,0 +1,295 @@
+#
+# 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.
+"""This module contains a Apache Beam Hook."""
+import json
+import select
+import shlex
+import subprocess
+import textwrap
+from tempfile import TemporaryDirectory
+from typing import Callable, List, Optional
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.python_virtualenv import prepare_virtualenv
+
+
+class BeamRunnerType:
+    """
+    Helper class for listing runner types.
+    For more information about runners see:
+    https://beam.apache.org/documentation/
+    """
+
+    DataflowRunner = "DataflowRunner"
+    DirectRunner = "DirectRunner"
+    SparkRunner = "SparkRunner"
+    FlinkRunner = "FlinkRunner"
+    SamzaRunner = "SamzaRunner"
+    NemoRunner = "NemoRunner"
+    JetRunner = "JetRunner"
+    Twister2Runner = "Twister2Runner"
+
+
+def beam_options_to_args(options: dict) -> List[str]:
+    """
+    Returns a formatted pipeline options from a dictionary of arguments
+
+    The logic of this method should be compatible with Apache Beam:
+    
https://github.com/apache/beam/blob/b56740f0e8cd80c2873412847d0b336837429fb9/sdks/python/
+    apache_beam/options/pipeline_options.py#L230-L251
+
+    :param options: Dictionary with options
+    :type options: dict
+    :return: List of arguments
+    :rtype: List[str]
+    """
+    if not options:
+        return []
+
+    args: List[str] = []
+    for attr, value in options.items():
+        if value is None or (isinstance(value, bool) and value):
+            args.append(f"--{attr}")
+        elif isinstance(value, list):
+            args.extend([f"--{attr}={v}" for v in value])
+        else:
+            args.append(f"--{attr}={value}")
+    return args
+
+
+class BeamCommandRunner(LoggingMixin):
+    """
+    Class responsible for running pipeline command in subprocess
+
+    :param cmd: Parts of the command to be run in subprocess
+    :type cmd: List[str]
+    :param process_line_callback: Optional callback which can be used to 
process
+        stdout and stderr to detect job id
+    :type process_line_callback: Optional[Callable[[str], None]]
+    """
+
+    def __init__(
+        self,
+        cmd: List[str],
+        process_line_callback: Optional[Callable[[str], None]] = None,
+    ) -> None:
+        super().__init__()
+        self.log.info("Running command: %s", " ".join(shlex.quote(c) for c in 
cmd))
+        self.process_line_callback = process_line_callback
+        self.job_id: Optional[str] = None
+        self._proc = subprocess.Popen(
+            cmd,
+            shell=False,
+            stdout=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+            close_fds=True,
+        )
+
+    def _process_fd(self, fd):
+        """
+        Prints output to logs.
+
+        :param fd: File descriptor.
+        """
+        if fd == self._proc.stderr:
+            while True:
+                line = self._proc.stderr.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.warning(line.rstrip("\n"))
+
+        if fd == self._proc.stdout:
+            while True:
+                line = self._proc.stdout.readline().decode()
+                if not line:
+                    return
+                if self.process_line_callback:
+                    self.process_line_callback(line)
+                self.log.info(line.rstrip("\n"))
+
+        raise Exception("No data in stderr or in stdout.")
+
+    def wait_for_done(self) -> None:
+        """Waits for Apache Beam pipeline to complete."""
+        self.log.info("Start waiting for Apache Beam process to complete.")
+        reads = [self._proc.stderr, self._proc.stdout]
+        while True:
+            # Wait for at least one available fd.
+            readable_fds, _, _ = select.select(reads, [], [], 5)
+            if readable_fds is None:
+                self.log.info("Waiting for Apache Beam process to complete.")
+                continue
+
+            for readable_fd in readable_fds:
+                self._process_fd(readable_fd)
+
+            if self._proc.poll() is not None:
+                break
+
+        # Corner case: check if more output was created between the last read 
and the process termination
+        for readable_fd in reads:
+            self._process_fd(readable_fd)
+
+        self.log.info("Process exited with return code: %s", 
self._proc.returncode)
+
+        if self._proc.returncode != 0:
+            raise Exception(f"Apache Beam process failed with return code 
{self._proc.returncode}")

Review comment:
       Could you explain why? I didn't see `SystemExit` in other hooks or 
operators. 




----------------------------------------------------------------
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:
[email protected]


Reply via email to