Repository: incubator-airflow Updated Branches: refs/heads/master 00fcd6891 -> f6f73b565
[AIRFLOW-1382] Add working dir option to DockerOperator Allow the user to specify the working directory to be used in the created container. Equivalent to docker run/create -w. Closes #2419 from benjaminsims/specify_working_directory Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/f6f73b56 Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/f6f73b56 Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/f6f73b56 Branch: refs/heads/master Commit: f6f73b565175118c15a6248408566fbb28a5736e Parents: 00fcd68 Author: Benjamin Sims <[email protected]> Authored: Tue Jul 11 20:56:31 2017 +0200 Committer: Bolke de Bruin <[email protected]> Committed: Tue Jul 11 20:56:37 2017 +0200 ---------------------------------------------------------------------- airflow/operators/docker_operator.py | 8 +++++++- tests/operators/docker_operator.py | 7 +++++-- 2 files changed, 12 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/f6f73b56/airflow/operators/docker_operator.py ---------------------------------------------------------------------- diff --git a/airflow/operators/docker_operator.py b/airflow/operators/docker_operator.py index bca604c..ddcc97b 100644 --- a/airflow/operators/docker_operator.py +++ b/airflow/operators/docker_operator.py @@ -70,6 +70,9 @@ class DockerOperator(BaseOperator): :type user: int or str :param volumes: List of volumes to mount into the container, e.g. ``['/host/path:/container/path', '/host/path2:/container/path2:ro']``. + :param working_dir: Working directory to set on the container (equivalent to the -w switch + the docker client) + :type working_dir: str :param xcom_push: Does the stdout will be pushed to the next step using XCom. The default is False. :type xcom_push: bool @@ -99,6 +102,7 @@ class DockerOperator(BaseOperator): tmp_dir='/tmp/airflow', user=None, volumes=None, + working_dir=None, xcom_push=False, xcom_all=False, *args, @@ -122,6 +126,7 @@ class DockerOperator(BaseOperator): self.tmp_dir = tmp_dir self.user = user self.volumes = volumes or [] + self.working_dir = working_dir self.xcom_push_flag = xcom_push self.xcom_all = xcom_all @@ -169,7 +174,8 @@ class DockerOperator(BaseOperator): network_mode=self.network_mode), image=image, mem_limit=self.mem_limit, - user=self.user + user=self.user, + working_dir=self.working_dir ) self.cli.start(self.container['Id']) http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/f6f73b56/tests/operators/docker_operator.py ---------------------------------------------------------------------- diff --git a/tests/operators/docker_operator.py b/tests/operators/docker_operator.py index 2554aaa..264096c 100644 --- a/tests/operators/docker_operator.py +++ b/tests/operators/docker_operator.py @@ -52,7 +52,8 @@ class DockerOperatorTestCase(unittest.TestCase): operator = DockerOperator(api_version='1.19', command='env', environment={'UNIT': 'TEST'}, image='ubuntu:latest', network_mode='bridge', owner='unittest', - task_id='unittest', volumes=['/host/path:/container/path']) + task_id='unittest', volumes=['/host/path:/container/path'], + working_dir='/container/path') operator.execute(None) client_class_mock.assert_called_with(base_url='unix://var/run/docker.sock', tls=None, @@ -65,7 +66,9 @@ class DockerOperatorTestCase(unittest.TestCase): }, host_config=host_config, image='ubuntu:latest', - mem_limit=None, user=None) + mem_limit=None, user=None, + working_dir='/container/path' + ) client_mock.create_host_config.assert_called_with(binds=['/host/path:/container/path', '/mkdtemp:/tmp/airflow'], network_mode='bridge')
