[1/5] incubator-airflow git commit: [AIRFLOW-219][AIRFLOW-398] Cgroups + impersonation
Repository: incubator-airflow Updated Branches: refs/heads/v1-8-test 8f9a466de -> 1387d3805 [AIRFLOW-219][AIRFLOW-398] Cgroups + impersonation Submitting on behalf of plypaul Please accept this PR that addresses the following issues: - https://issues.apache.org/jira/browse/AIRFLOW-219 - https://issues.apache.org/jira/browse/AIRFLOW-398 Testing Done: - Running on Airbnb prod (though on a different mergebase) for many months Credits: Impersonation Work: georgeke did most of the work but plypaul did quite a bit of work too. Cgroups: plypaul did most of the work, I just did some touch up/bug fixes (see commit history, cgroups + impersonation commit is actually plypaul 's not mine) Closes #1934 from aoen/ddavydov/cgroups_and_impers onation_after_rebase Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/b56cb5cc Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/b56cb5cc Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/b56cb5cc Branch: refs/heads/v1-8-test Commit: b56cb5cc97de074bb0e520f66b79e7eb2d913fb1 Parents: 8f9a466 Author: Dan DavydovAuthored: Wed Jan 18 18:11:01 2017 -0800 Committer: Dan Davydov Committed: Wed Jan 18 18:11:06 2017 -0800 -- .travis.yml | 2 +- airflow/bin/cli.py | 96 +++-- airflow/configuration.py| 7 + airflow/contrib/task_runner/__init__.py | 13 ++ .../contrib/task_runner/cgroup_task_runner.py | 202 +++ airflow/jobs.py | 67 +++--- .../1a5a9e6bf2b5_add_state_index_for_dagruns.py | 37 airflow/models.py | 92 ++--- airflow/settings.py | 23 ++- airflow/task_runner/__init__.py | 38 airflow/task_runner/base_task_runner.py | 153 ++ airflow/task_runner/bash_task_runner.py | 39 airflow/utils/file.py | 23 +++ airflow/utils/helpers.py| 79 +++- docs/security.rst | 22 ++ run_unit_tests.sh | 14 ++ scripts/ci/airflow_travis.cfg | 1 + scripts/ci/requirements.txt | 1 + setup.py| 4 + tests/__init__.py | 1 + tests/dags/test_default_impersonation.py| 44 tests/dags/test_impersonation.py| 45 + tests/dags/test_no_impersonation.py | 43 tests/impersonation.py | 111 ++ 24 files changed, 1061 insertions(+), 96 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/b56cb5cc/.travis.yml -- diff --git a/.travis.yml b/.travis.yml index 407e7f9..90f33e3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -89,7 +89,7 @@ cache: - $HOME/.wheelhouse/ - $HOME/.travis_cache/ before_install: - - ssh-keygen -t rsa -C your_em...@youremail.com -P '' -f ~/.ssh/id_rsa + - yes | ssh-keygen -t rsa -C your_em...@youremail.com -P '' -f ~/.ssh/id_rsa - cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys - ln -s ~/.ssh/authorized_keys ~/.ssh/authorized_keys2 - chmod 600 ~/.ssh/* http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/b56cb5cc/airflow/bin/cli.py -- diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index d55fdfc..736df0a 100755 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -22,7 +22,6 @@ import os import subprocess import textwrap import warnings -from datetime import datetime from importlib import import_module import argparse @@ -53,7 +52,7 @@ from airflow.models import (DagModel, DagBag, TaskInstance, from airflow.ti_deps.dep_context import (DepContext, SCHEDULER_DEPS) from airflow.utils import db as db_utils from airflow.utils import logging as logging_utils -from airflow.utils.state import State +from airflow.utils.file import mkdirs from airflow.www.app import cached_app from sqlalchemy import func @@ -300,6 +299,7 @@ def export_helper(filepath): varfile.write(json.dumps(var_dict, sort_keys=True, indent=4)) print("{} variables successfully exported to {}".format(len(var_dict), filepath)) + def pause(args, dag=None): set_is_paused(True, args, dag) @@ -329,19 +329,65 @@ def run(args, dag=None): if dag: args.dag_id = dag.dag_id -# Setting up logging -log_base = os.path.expanduser(conf.get('core', 'BASE_LOG_FOLDER')) -directory =
incubator-airflow git commit: [AIRFLOW-219][AIRFLOW-398] Cgroups + impersonation
Repository: incubator-airflow Updated Branches: refs/heads/master 8f9a466de -> b56cb5cc9 [AIRFLOW-219][AIRFLOW-398] Cgroups + impersonation Submitting on behalf of plypaul Please accept this PR that addresses the following issues: - https://issues.apache.org/jira/browse/AIRFLOW-219 - https://issues.apache.org/jira/browse/AIRFLOW-398 Testing Done: - Running on Airbnb prod (though on a different mergebase) for many months Credits: Impersonation Work: georgeke did most of the work but plypaul did quite a bit of work too. Cgroups: plypaul did most of the work, I just did some touch up/bug fixes (see commit history, cgroups + impersonation commit is actually plypaul 's not mine) Closes #1934 from aoen/ddavydov/cgroups_and_impers onation_after_rebase Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/b56cb5cc Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/b56cb5cc Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/b56cb5cc Branch: refs/heads/master Commit: b56cb5cc97de074bb0e520f66b79e7eb2d913fb1 Parents: 8f9a466 Author: Dan DavydovAuthored: Wed Jan 18 18:11:01 2017 -0800 Committer: Dan Davydov Committed: Wed Jan 18 18:11:06 2017 -0800 -- .travis.yml | 2 +- airflow/bin/cli.py | 96 +++-- airflow/configuration.py| 7 + airflow/contrib/task_runner/__init__.py | 13 ++ .../contrib/task_runner/cgroup_task_runner.py | 202 +++ airflow/jobs.py | 67 +++--- .../1a5a9e6bf2b5_add_state_index_for_dagruns.py | 37 airflow/models.py | 92 ++--- airflow/settings.py | 23 ++- airflow/task_runner/__init__.py | 38 airflow/task_runner/base_task_runner.py | 153 ++ airflow/task_runner/bash_task_runner.py | 39 airflow/utils/file.py | 23 +++ airflow/utils/helpers.py| 79 +++- docs/security.rst | 22 ++ run_unit_tests.sh | 14 ++ scripts/ci/airflow_travis.cfg | 1 + scripts/ci/requirements.txt | 1 + setup.py| 4 + tests/__init__.py | 1 + tests/dags/test_default_impersonation.py| 44 tests/dags/test_impersonation.py| 45 + tests/dags/test_no_impersonation.py | 43 tests/impersonation.py | 111 ++ 24 files changed, 1061 insertions(+), 96 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/b56cb5cc/.travis.yml -- diff --git a/.travis.yml b/.travis.yml index 407e7f9..90f33e3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -89,7 +89,7 @@ cache: - $HOME/.wheelhouse/ - $HOME/.travis_cache/ before_install: - - ssh-keygen -t rsa -C your_em...@youremail.com -P '' -f ~/.ssh/id_rsa + - yes | ssh-keygen -t rsa -C your_em...@youremail.com -P '' -f ~/.ssh/id_rsa - cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys - ln -s ~/.ssh/authorized_keys ~/.ssh/authorized_keys2 - chmod 600 ~/.ssh/* http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/b56cb5cc/airflow/bin/cli.py -- diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index d55fdfc..736df0a 100755 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -22,7 +22,6 @@ import os import subprocess import textwrap import warnings -from datetime import datetime from importlib import import_module import argparse @@ -53,7 +52,7 @@ from airflow.models import (DagModel, DagBag, TaskInstance, from airflow.ti_deps.dep_context import (DepContext, SCHEDULER_DEPS) from airflow.utils import db as db_utils from airflow.utils import logging as logging_utils -from airflow.utils.state import State +from airflow.utils.file import mkdirs from airflow.www.app import cached_app from sqlalchemy import func @@ -300,6 +299,7 @@ def export_helper(filepath): varfile.write(json.dumps(var_dict, sort_keys=True, indent=4)) print("{} variables successfully exported to {}".format(len(var_dict), filepath)) + def pause(args, dag=None): set_is_paused(True, args, dag) @@ -329,19 +329,65 @@ def run(args, dag=None): if dag: args.dag_id = dag.dag_id -# Setting up logging -log_base = os.path.expanduser(conf.get('core', 'BASE_LOG_FOLDER')) -directory =