ashb commented on a change in pull request #15330: URL: https://github.com/apache/airflow/pull/15330#discussion_r614327894
########## File path: airflow/decorators/docker.py ########## @@ -0,0 +1,159 @@ +# 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. + +import base64 +import inspect +import os +import pickle +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import Callable, Dict, Optional, TypeVar + +from airflow.decorators.base import DecoratedOperator, task_decorator_factory +from airflow.providers.docker.operators.docker import DockerOperator +from airflow.utils.decorators import apply_defaults +from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script + + +def _generate_decode_command(env_var, file): + return ( + f'python -c "import os; import base64;' + f' x = base64.b64decode(os.environ[\\"{env_var}\\"]);' + f' f = open(\\"{file}\\", \\"wb\\"); f.write(x);' + f' f.close()"' + ) + + +def _b64_encode_file(filename): + data = open(filename, "rb").read() + encoded = base64.b64encode(data) + return encoded + + +class _DockerDecoratedOperator(DecoratedOperator, DockerOperator): + """ + Wraps a Python callable and captures args/kwargs when called for execution. + + :param python_callable: A reference to an object that is callable + :type python_callable: python callable + :param op_kwargs: a dictionary of keyword arguments that will get unpacked + in your function (templated) + :type op_kwargs: dict + :param op_args: a list of positional arguments that will get unpacked when + calling your callable (templated) + :type op_args: list + :param multiple_outputs: if set, function return value will be + unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys. + Defaults to False. + :type multiple_outputs: bool + """ + + template_fields = ('op_args', 'op_kwargs') + template_fields_renderers = {"op_args": "py", "op_kwargs": "py"} + + # since we won't mutate the arguments, we should just do the shallow copy + # there are some cases we can't deepcopy the objects (e.g protobuf). + shallow_copy_attrs = ('python_callable',) + + @apply_defaults + def __init__( + self, + **kwargs, + ) -> None: + self.string_args = [1, 2, 1] + self._output_filename = "" + command = "dummy command" + self.pickling_library = pickle + super().__init__( + command=command, retrieve_output=True, retrieve_output_path="/tmp/script.out", **kwargs + ) + + def execute(self, context: Dict): + with TemporaryDirectory(prefix='venv') as tmp_dir: + input_filename = os.path.join(tmp_dir, 'script.in') + self._output_filename = os.path.join(tmp_dir, 'script.out') + string_args_filename = os.path.join(tmp_dir, 'string_args.txt') + script_filename = os.path.join(tmp_dir, 'script.py') + self._write_args(input_filename) + self._write_string_args(string_args_filename) + py_source = self._get_python_source() + write_python_script( + jinja_context=dict( + op_args=self.op_args, + op_kwargs=self.op_kwargs, + pickling_library=self.pickling_library.__name__, + python_callable=self.python_callable.__name__, + python_callable_source=py_source, + ), + filename=script_filename, + ) + + self.environment["PYTHON_SCRIPT"] = _b64_encode_file(script_filename) + if self.op_args or self.op_kwargs: + self.environment["PYTHON_INPUT"] = _b64_encode_file(input_filename) + else: + self.environment["PYTHON_INPUT"] = "" + + self.command = ( + f'bash -cx \'{_generate_decode_command("PYTHON_SCRIPT", "/tmp/script.py")} &&' + f'touch /tmp/string_args &&' + f'touch /tmp/script.in &&' + f'{_generate_decode_command("PYTHON_INPUT", "/tmp/script.in")} &&' + f'python /tmp/script.py /tmp/script.in /tmp/script.out /tmp/string_args \'' + ) + return super().execute(context) + + def _write_args(self, filename): + if self.op_args or self.op_kwargs: + with open(filename, 'wb') as file: + self.pickling_library.dump({'args': self.op_args, 'kwargs': self.op_kwargs}, file) + + def _write_string_args(self, filename): + with open(filename, 'w') as file: + file.write('\n'.join(map(str, self.string_args))) Review comment: Two things here: 1. This is duplicating code from VenvOperator, we should fix a way to not need that (make a mixin perhaps?) 2. The whole `self.string_args` is a misfeature of Venv we shouldn't copy over anyway. ########## File path: airflow/decorators/docker.py ########## @@ -0,0 +1,159 @@ +# 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. + +import base64 +import inspect +import os +import pickle +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import Callable, Dict, Optional, TypeVar + +from airflow.decorators.base import DecoratedOperator, task_decorator_factory +from airflow.providers.docker.operators.docker import DockerOperator +from airflow.utils.decorators import apply_defaults +from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script + + +def _generate_decode_command(env_var, file): + return ( + f'python -c "import os; import base64;' + f' x = base64.b64decode(os.environ[\\"{env_var}\\"]);' + f' f = open(\\"{file}\\", \\"wb\\"); f.write(x);' + f' f.close()"' + ) Review comment: ```suggestion return shlex.quote( f'python -c "import os; import base64;' f' x = base64.b64decode(os.environ["{env_var}"]);' f' f = open("{file}", "wb"); f.write(x);' f' f.close()"' ) ``` ########## File path: airflow/decorators/docker.py ########## @@ -0,0 +1,159 @@ +# 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. + +import base64 +import inspect +import os +import pickle +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import Callable, Dict, Optional, TypeVar + +from airflow.decorators.base import DecoratedOperator, task_decorator_factory +from airflow.providers.docker.operators.docker import DockerOperator +from airflow.utils.decorators import apply_defaults +from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script + + +def _generate_decode_command(env_var, file): + return ( + f'python -c "import os; import base64;' + f' x = base64.b64decode(os.environ[\\"{env_var}\\"]);' + f' f = open(\\"{file}\\", \\"wb\\"); f.write(x);' + f' f.close()"' + ) + + +def _b64_encode_file(filename): + data = open(filename, "rb").read() + encoded = base64.b64encode(data) + return encoded Review comment: "Leaking" a FH here (not strictly, as it'll be closed when GC'd) ```suggestion with open(filename, "rb") as fh: return base64.b64encode(fh.read()) ``` ########## File path: airflow/decorators/docker.py ########## @@ -0,0 +1,159 @@ +# 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. + +import base64 +import inspect +import os +import pickle +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import Callable, Dict, Optional, TypeVar + +from airflow.decorators.base import DecoratedOperator, task_decorator_factory +from airflow.providers.docker.operators.docker import DockerOperator +from airflow.utils.decorators import apply_defaults +from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script + + +def _generate_decode_command(env_var, file): + return ( + f'python -c "import os; import base64;' + f' x = base64.b64decode(os.environ[\\"{env_var}\\"]);' + f' f = open(\\"{file}\\", \\"wb\\"); f.write(x);' + f' f.close()"' + ) + + +def _b64_encode_file(filename): + data = open(filename, "rb").read() + encoded = base64.b64encode(data) + return encoded + + +class _DockerDecoratedOperator(DecoratedOperator, DockerOperator): + """ + Wraps a Python callable and captures args/kwargs when called for execution. + + :param python_callable: A reference to an object that is callable + :type python_callable: python callable + :param op_kwargs: a dictionary of keyword arguments that will get unpacked + in your function (templated) + :type op_kwargs: dict + :param op_args: a list of positional arguments that will get unpacked when + calling your callable (templated) + :type op_args: list + :param multiple_outputs: if set, function return value will be + unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys. + Defaults to False. + :type multiple_outputs: bool + """ + + template_fields = ('op_args', 'op_kwargs') + template_fields_renderers = {"op_args": "py", "op_kwargs": "py"} + + # since we won't mutate the arguments, we should just do the shallow copy + # there are some cases we can't deepcopy the objects (e.g protobuf). + shallow_copy_attrs = ('python_callable',) + + @apply_defaults + def __init__( + self, + **kwargs, + ) -> None: + self.string_args = [1, 2, 1] + self._output_filename = "" + command = "dummy command" + self.pickling_library = pickle + super().__init__( + command=command, retrieve_output=True, retrieve_output_path="/tmp/script.out", **kwargs + ) + + def execute(self, context: Dict): + with TemporaryDirectory(prefix='venv') as tmp_dir: + input_filename = os.path.join(tmp_dir, 'script.in') + self._output_filename = os.path.join(tmp_dir, 'script.out') + string_args_filename = os.path.join(tmp_dir, 'string_args.txt') + script_filename = os.path.join(tmp_dir, 'script.py') + self._write_args(input_filename) + self._write_string_args(string_args_filename) + py_source = self._get_python_source() + write_python_script( + jinja_context=dict( + op_args=self.op_args, + op_kwargs=self.op_kwargs, + pickling_library=self.pickling_library.__name__, + python_callable=self.python_callable.__name__, + python_callable_source=py_source, + ), + filename=script_filename, + ) + + self.environment["PYTHON_SCRIPT"] = _b64_encode_file(script_filename) + if self.op_args or self.op_kwargs: + self.environment["PYTHON_INPUT"] = _b64_encode_file(input_filename) + else: + self.environment["PYTHON_INPUT"] = "" + + self.command = ( + f'bash -cx \'{_generate_decode_command("PYTHON_SCRIPT", "/tmp/script.py")} &&' + f'touch /tmp/string_args &&' + f'touch /tmp/script.in &&' + f'{_generate_decode_command("PYTHON_INPUT", "/tmp/script.in")} &&' + f'python /tmp/script.py /tmp/script.in /tmp/script.out /tmp/string_args \'' Review comment: Why bash _and_ python -- why not just do this all in the one generated python command? ########## File path: airflow/example_dags/tutorial_taskflow_api_etl_docker_virtualenv.py ########## @@ -0,0 +1,120 @@ +# +# 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. + +# pylint: disable=missing-function-docstring + +# [START tutorial] +# [START import_module] +import json + +from airflow.decorators import dag, task +from airflow.utils.dates import days_ago + +# [END import_module] + +# [START default_args] +# These args will get passed on to each operator +# You can override them on a per-task basis during operator initialization +default_args = { + 'owner': 'airflow', +} +# [END default_args] + + +# [START instantiate_dag] +@dag(default_args=default_args, schedule_interval=None, start_date=days_ago(2), tags=['example']) +def tutorial_taskflow_api_etl(): + """ + ### TaskFlow API Tutorial Documentation + This is a simple ETL data pipeline example which demonstrates the use of + the TaskFlow API using three simple tasks for Extract, Transform, and Load. + Documentation that goes along with the Airflow TaskFlow API tutorial is + located + [here](https://airflow.apache.org/docs/apache-airflow/stable/tutorial_taskflow_api.html) + """ + # [END instantiate_dag] + + # [START extract_virtualenv] + @task.virtualenv( + use_dill=True, + system_site_packages=False, + requirements=['funcsigs'], + ) + def extract(): + """ + #### Extract task + A simple Extract task to get data ready for the rest of the data + pipeline. In this case, getting data is simulated by reading from a + hardcoded JSON string. + """ + data_string = '{"1001": 301.27, "1002": 433.21, "1003": 502.22}' + + order_data_dict = json.loads(data_string) + return order_data_dict + + # [END extract_virtualenv] + + # [START transform_docker] + @task.docker( + image="python:3.8.8-slim-buster", + force_pull=True, + docker_url="unix://var/run/docker.sock", + network_mode="bridge", + api_version='auto', Review comment: ```suggestion ``` Those are all the defaults, so we shouldn't need to include them in the example. -- 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]
