You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by cr...@apache.org on 2017/09/13 17:39:45 UTC

[01/37] incubator-airflow git commit: [AIRFLOW-1273] Add Google Cloud ML version and model operators

Repository: incubator-airflow
Updated Branches:
  refs/heads/master 653562e68 -> a8d486de3


[AIRFLOW-1273] Add Google Cloud ML version and model operators

https://issues.apache.org/jira/browse/AIRFLOW-1273

Closes #2379 from N3da/master


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/265b293a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/265b293a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/265b293a

Branch: refs/heads/master
Commit: 265b293a7860d070458b6984594138bfae1fa5b7
Parents: e870a8e
Author: Neda Mirian <ne...@google.com>
Authored: Tue Jun 27 09:39:00 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Tue Jun 27 09:39:18 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/__init__.py             |   1 +
 airflow/contrib/hooks/gcp_cloudml_hook.py     | 167 ++++++++++++++
 airflow/contrib/operators/cloudml_operator.py | 178 ++++++++++++++
 airflow/utils/db.py                           |   4 +
 tests/contrib/hooks/test_gcp_cloudml_hook.py  | 255 +++++++++++++++++++++
 5 files changed, 605 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/265b293a/airflow/contrib/hooks/__init__.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/__init__.py b/airflow/contrib/hooks/__init__.py
index 182a49f..4941314 100644
--- a/airflow/contrib/hooks/__init__.py
+++ b/airflow/contrib/hooks/__init__.py
@@ -40,6 +40,7 @@ _hooks = {
     'qubole_hook': ['QuboleHook'],
     'gcs_hook': ['GoogleCloudStorageHook'],
     'datastore_hook': ['DatastoreHook'],
+    'gcp_cloudml_hook': ['CloudMLHook'],
     'gcp_dataproc_hook': ['DataProcHook'],
     'gcp_dataflow_hook': ['DataFlowHook'],
     'spark_submit_operator': ['SparkSubmitOperator'],

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/265b293a/airflow/contrib/hooks/gcp_cloudml_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/gcp_cloudml_hook.py b/airflow/contrib/hooks/gcp_cloudml_hook.py
new file mode 100644
index 0000000..e722b2a
--- /dev/null
+++ b/airflow/contrib/hooks/gcp_cloudml_hook.py
@@ -0,0 +1,167 @@
+#
+# 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 logging
+import random
+import time
+from airflow import settings
+from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
+from apiclient.discovery import build
+from apiclient import errors
+from oauth2client.client import GoogleCredentials
+
+logging.getLogger('GoogleCloudML').setLevel(settings.LOGGING_LEVEL)
+
+
+def _poll_with_exponential_delay(request, max_n, is_done_func, is_error_func):
+
+    for i in range(0, max_n):
+        try:
+            response = request.execute()
+            if is_error_func(response):
+                raise ValueError('The response contained an error: {}'.format(response))
+            elif is_done_func(response):
+                logging.info('Operation is done: {}'.format(response))
+                return response
+            else:
+                time.sleep((2**i) + (random.randint(0, 1000) / 1000))
+        except errors.HttpError as e:
+            if e.resp.status != 429:
+                logging.info('Something went wrong. Not retrying: {}'.format(e))
+                raise e
+            else:
+                time.sleep((2**i) + (random.randint(0, 1000) / 1000))
+
+
+class CloudMLHook(GoogleCloudBaseHook):
+
+    def __init__(self, gcp_conn_id='google_cloud_default', delegate_to=None):
+        super(CloudMLHook, self).__init__(gcp_conn_id, delegate_to)
+        self._cloudml = self.get_conn()
+
+    def get_conn(self):
+        """
+        Returns a Google CloudML service object.
+        """
+        credentials = GoogleCredentials.get_application_default()
+        return build('ml', 'v1', credentials=credentials)
+
+    def create_version(self, project_name, model_name, version_spec):
+        """
+        Creates the Version on Cloud ML.
+
+        Returns the operation if the version was created successfully and raises
+        an error otherwise.
+        """
+        parent_name = 'projects/{}/models/{}'.format(project_name, model_name)
+        create_request = self._cloudml.projects().models().versions().create(
+            parent=parent_name, body=version_spec)
+        response = create_request.execute()
+        get_request = self._cloudml.projects().operations().get(
+            name=response['name'])
+
+        return _poll_with_exponential_delay(
+            request=get_request,
+            max_n=9,
+            is_done_func=lambda resp: resp.get('done', False),
+            is_error_func=lambda resp: resp.get('error', None) is not None)
+
+    def set_default_version(self, project_name, model_name, version_name):
+        """
+        Sets a version to be the default. Blocks until finished.
+        """
+        full_version_name = 'projects/{}/models/{}/versions/{}'.format(
+            project_name, model_name, version_name)
+        request = self._cloudml.projects().models().versions().setDefault(
+            name=full_version_name, body={})
+
+        try:
+            response = request.execute()
+            logging.info('Successfully set version: {} to default'.format(response))
+            return response
+        except errors.HttpError as e:
+            logging.error('Something went wrong: {}'.format(e))
+            raise e
+
+    def list_versions(self, project_name, model_name):
+        """
+        Lists all available versions of a model. Blocks until finished.
+        """
+        result = []
+        full_parent_name = 'projects/{}/models/{}'.format(
+            project_name, model_name)
+        request = self._cloudml.projects().models().versions().list(
+            parent=full_parent_name, pageSize=100)
+
+        response = request.execute()
+        next_page_token = response.get('nextPageToken', None)
+        result.extend(response.get('versions', []))
+        while next_page_token is not None:
+            next_request = self._cloudml.projects().models().versions().list(
+                parent=full_parent_name,
+                pageToken=next_page_token,
+                pageSize=100)
+            response = next_request.execute()
+            next_page_token = response.get('nextPageToken', None)
+            result.extend(response.get('versions', []))
+            time.sleep(5)
+        return result
+
+    def delete_version(self, project_name, model_name, version_name):
+        """
+        Deletes the given version of a model. Blocks until finished.
+        """
+        full_name = 'projects/{}/models/{}/versions/{}'.format(
+            project_name, model_name, version_name)
+        delete_request = self._cloudml.projects().models().versions().delete(
+            name=full_name)
+        response = delete_request.execute()
+        get_request = self._cloudml.projects().operations().get(
+            name=response['name'])
+
+        return _poll_with_exponential_delay(
+            request=get_request,
+            max_n=9,
+            is_done_func=lambda resp: resp.get('done', False),
+            is_error_func=lambda resp: resp.get('error', None) is not None)
+
+    def create_model(self, project_name, model):
+        """
+        Create a Model. Blocks until finished.
+        """
+        assert model['name'] is not None and model['name'] is not ''
+        project = 'projects/{}'.format(project_name)
+
+        request = self._cloudml.projects().models().create(
+            parent=project, body=model)
+        return request.execute()
+
+    def get_model(self, project_name, model_name):
+        """
+        Gets a Model. Blocks until finished.
+        """
+        assert model_name is not None and model_name is not ''
+        full_model_name = 'projects/{}/models/{}'.format(
+            project_name, model_name)
+        request = self._cloudml.projects().models().get(name=full_model_name)
+        try:
+            return request.execute()
+        except errors.HttpError as e:
+            if e.resp.status == 404:
+                logging.error('Model was not found: {}'.format(e))
+                return None
+            raise e

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/265b293a/airflow/contrib/operators/cloudml_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/cloudml_operator.py b/airflow/contrib/operators/cloudml_operator.py
new file mode 100644
index 0000000..b0b6e91
--- /dev/null
+++ b/airflow/contrib/operators/cloudml_operator.py
@@ -0,0 +1,178 @@
+#
+# 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 logging
+from airflow import settings
+from airflow.contrib.hooks.gcp_cloudml_hook import CloudMLHook
+from airflow.operators import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+logging.getLogger('GoogleCloudML').setLevel(settings.LOGGING_LEVEL)
+
+
+class CloudMLVersionOperator(BaseOperator):
+    """
+    Operator for managing a Google Cloud ML version.
+
+    :param model_name: The name of the Google Cloud ML model that the version
+        belongs to.
+    :type model_name: string
+
+    :param project_name: The Google Cloud project name to which CloudML
+        model belongs.
+    :type project_name: string
+
+    :param version: A dictionary containing the information about the version.
+        If the `operation` is `create`, `version` should contain all the
+        information about this version such as name, and deploymentUrl.
+        If the `operation` is `get` or `delete`, the `version` parameter
+        should contain the `name` of the version.
+        If it is None, the only `operation` possible would be `list`.
+    :type version: dict
+
+    :param gcp_conn_id: The connection ID to use when fetching connection info.
+    :type gcp_conn_id: string
+
+    :param operation: The operation to perform. Available operations are:
+        'create': Creates a new version in the model specified by `model_name`,
+            in which case the `version` parameter should contain all the
+            information to create that version
+            (e.g. `name`, `deploymentUrl`).
+        'get': Gets full information of a particular version in the model
+            specified by `model_name`.
+            The name of the version should be specified in the `version`
+            parameter.
+
+        'list': Lists all available versions of the model specified
+            by `model_name`.
+
+        'delete': Deletes the version specified in `version` parameter from the
+            model specified by `model_name`).
+            The name of the version should be specified in the `version`
+            parameter.
+     :type operation: string
+
+    :param delegate_to: The account to impersonate, if any.
+        For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: string
+    """
+
+
+    template_fields = [
+        '_model_name',
+        '_version',
+    ]
+
+    @apply_defaults
+    def __init__(self,
+                 model_name,
+                 project_name,
+                 version=None,
+                 gcp_conn_id='google_cloud_default',
+                 operation='create',
+                 delegate_to=None,
+                 *args,
+                 **kwargs):
+
+        super(CloudMLVersionOperator, self).__init__(*args, **kwargs)
+        self._model_name = model_name
+        self._version = version
+        self._gcp_conn_id = gcp_conn_id
+        self._delegate_to = delegate_to
+        self._project_name = project_name
+        self._operation = operation
+
+    def execute(self, context):
+        hook = CloudMLHook(
+            gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+
+        if self._operation == 'create':
+            assert self._version is not None
+            return hook.create_version(self._project_name, self._model_name,
+                                       self._version)
+        elif self._operation == 'set_default':
+            return hook.set_default_version(
+                self._project_name, self._model_name,
+                self._version['name'])
+        elif self._operation == 'list':
+            return hook.list_versions(self._project_name, self._model_name)
+        elif self._operation == 'delete':
+            return hook.delete_version(self._project_name, self._model_name,
+                                       self._version['name'])
+        else:
+            raise ValueError('Unknown operation: {}'.format(self._operation))
+
+
+class CloudMLModelOperator(BaseOperator):
+    """
+    Operator for managing a Google Cloud ML model.
+
+    :param model: A dictionary containing the information about the model.
+        If the `operation` is `create`, then the `model` parameter should
+        contain all the information about this model such as `name`.
+
+        If the `operation` is `get`, the `model` parameter
+        should contain the `name` of the model.
+    :type model: dict
+
+    :param project_name: The Google Cloud project name to which CloudML
+        model belongs.
+    :type project_name: string
+
+    :param gcp_conn_id: The connection ID to use when fetching connection info.
+    :type gcp_conn_id: string
+
+    :param operation: The operation to perform. Available operations are:
+        'create': Creates a new model as provided by the `model` parameter.
+        'get': Gets a particular model where the name is specified in `model`.
+
+    :param delegate_to: The account to impersonate, if any.
+        For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: string
+    """
+
+    template_fields = [
+        '_model',
+    ]
+
+    @apply_defaults
+    def __init__(self,
+                 model,
+                 project_name,
+                 gcp_conn_id='google_cloud_default',
+                 operation='create',
+                 delegate_to=None,
+                 *args,
+                 **kwargs):
+        super(CloudMLModelOperator, self).__init__(*args, **kwargs)
+        self._model = model
+        self._operation = operation
+        self._gcp_conn_id = gcp_conn_id
+        self._delegate_to = delegate_to
+        self._project_name = project_name
+
+    def execute(self, context):
+        hook = CloudMLHook(
+            gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        if self._operation == 'create':
+            hook.create_model(self._project_name, self._model)
+        elif self._operation == 'get':
+            hook.get_model(self._project_name, self._model['name'])
+        else:
+            raise ValueError('Unknown operation: {}'.format(self._operation))

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/265b293a/airflow/utils/db.py
----------------------------------------------------------------------
diff --git a/airflow/utils/db.py b/airflow/utils/db.py
index 54254f6..04b1512 100644
--- a/airflow/utils/db.py
+++ b/airflow/utils/db.py
@@ -130,6 +130,10 @@ def initdb():
             schema='hive', port=3400))
     merge_conn(
         models.Connection(
+            conn_id='google_cloud_default', conn_type='google_cloud_platform',
+            schema='default',))
+    merge_conn(
+        models.Connection(
             conn_id='hive_cli_default', conn_type='hive_cli',
             schema='default',))
     merge_conn(

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/265b293a/tests/contrib/hooks/test_gcp_cloudml_hook.py
----------------------------------------------------------------------
diff --git a/tests/contrib/hooks/test_gcp_cloudml_hook.py b/tests/contrib/hooks/test_gcp_cloudml_hook.py
new file mode 100644
index 0000000..aa50e69
--- /dev/null
+++ b/tests/contrib/hooks/test_gcp_cloudml_hook.py
@@ -0,0 +1,255 @@
+# Licensed 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 json
+import mock
+import unittest
+try: # python 2
+    from urlparse import urlparse, parse_qsl
+except ImportError: #python 3
+    from urllib.parse import urlparse, parse_qsl
+
+from airflow.contrib.hooks import gcp_cloudml_hook as hook
+from apiclient.discovery import build
+from apiclient.http import HttpMockSequence
+from oauth2client.contrib.gce import HttpAccessTokenRefreshError
+
+cml_available = True
+try:
+    hook.CloudMLHook().get_conn()
+except HttpAccessTokenRefreshError:
+    cml_available = False
+
+
+class _TestCloudMLHook(object):
+
+    def __init__(self, test_cls, responses, expected_requests):
+        """
+        Init method.
+
+        Usage example:
+        with _TestCloudMLHook(self, responses, expected_requests) as hook:
+            self.run_my_test(hook)
+
+        Args:
+          test_cls: The caller's instance used for test communication.
+          responses: A list of (dict_response, response_content) tuples.
+          expected_requests: A list of (uri, http_method, body) tuples.
+        """
+
+        self._test_cls = test_cls
+        self._responses = responses
+        self._expected_requests = [
+            self._normalize_requests_for_comparison(x[0], x[1], x[2]) for x in expected_requests]
+        self._actual_requests = []
+
+    def _normalize_requests_for_comparison(self, uri, http_method, body):
+        parts = urlparse(uri)
+        return (parts._replace(query=set(parse_qsl(parts.query))), http_method, body)
+
+    def __enter__(self):
+        http = HttpMockSequence(self._responses)
+        native_request_method = http.request
+
+        # Collecting requests to validate at __exit__.
+        def _request_wrapper(*args, **kwargs):
+            self._actual_requests.append(args + (kwargs['body'],))
+            return native_request_method(*args, **kwargs)
+
+        http.request = _request_wrapper
+        service_mock = build('ml', 'v1', http=http)
+        with mock.patch.object(
+                hook.CloudMLHook, 'get_conn', return_value=service_mock):
+            return hook.CloudMLHook()
+
+    def __exit__(self, *args):
+        # Propogating exceptions here since assert will silence them.
+        if any(args):
+            return None
+        self._test_cls.assertEquals(
+            [self._normalize_requests_for_comparison(x[0], x[1], x[2]) for x in self._actual_requests], self._expected_requests)
+
+
+class TestCloudMLHook(unittest.TestCase):
+
+    def setUp(self):
+        pass
+
+    _SKIP_IF = unittest.skipIf(not cml_available,
+                               'CloudML is not available to run tests')
+    _SERVICE_URI_PREFIX = 'https://ml.googleapis.com/v1/'
+
+    @_SKIP_IF
+    def test_create_version(self):
+        project = 'test-project'
+        model_name = 'test-model'
+        version = 'test-version'
+        operation_name = 'projects/{}/operations/test-operation'.format(
+            project)
+
+        response_body = {'name': operation_name, 'done': True}
+        succeeded_response = ({'status': '200'}, json.dumps(response_body))
+
+        expected_requests = [
+            ('{}projects/{}/models/{}/versions?alt=json'.format(
+                self._SERVICE_URI_PREFIX, project, model_name), 'POST',
+             '"{}"'.format(version)),
+            ('{}{}?alt=json'.format(self._SERVICE_URI_PREFIX, operation_name),
+             'GET', None),
+        ]
+
+        with _TestCloudMLHook(
+                self,
+                responses=[succeeded_response] * 2,
+                expected_requests=expected_requests) as cml_hook:
+            create_version_response = cml_hook.create_version(
+                project_name=project, model_name=model_name, version_spec=version)
+            self.assertEquals(create_version_response, response_body)
+
+    @_SKIP_IF
+    def test_set_default_version(self):
+        project = 'test-project'
+        model_name = 'test-model'
+        version = 'test-version'
+        operation_name = 'projects/{}/operations/test-operation'.format(
+            project)
+
+        response_body = {'name': operation_name, 'done': True}
+        succeeded_response = ({'status': '200'}, json.dumps(response_body))
+
+        expected_requests = [
+            ('{}projects/{}/models/{}/versions/{}:setDefault?alt=json'.format(
+                self._SERVICE_URI_PREFIX, project, model_name, version), 'POST',
+             '{}'),
+        ]
+
+        with _TestCloudMLHook(
+                self,
+                responses=[succeeded_response],
+                expected_requests=expected_requests) as cml_hook:
+            set_default_version_response = cml_hook.set_default_version(
+                project_name=project, model_name=model_name, version_name=version)
+            self.assertEquals(set_default_version_response, response_body)
+
+    @_SKIP_IF
+    def test_list_versions(self):
+        project = 'test-project'
+        model_name = 'test-model'
+        operation_name = 'projects/{}/operations/test-operation'.format(
+            project)
+
+        # This test returns the versions one at a time.
+        versions = ['ver_{}'.format(ix) for ix in range(3)]
+
+        response_bodies = [{'name': operation_name, 'nextPageToken': ix, 'versions': [
+            ver]} for ix, ver in enumerate(versions)]
+        response_bodies[-1].pop('nextPageToken')
+        responses = [({'status': '200'}, json.dumps(body))
+                     for body in response_bodies]
+
+        expected_requests = [
+            ('{}projects/{}/models/{}/versions?alt=json&pageSize=100'.format(
+                self._SERVICE_URI_PREFIX, project, model_name), 'GET',
+             None),
+        ] + [
+            ('{}projects/{}/models/{}/versions?alt=json&pageToken={}&pageSize=100'.format(
+                self._SERVICE_URI_PREFIX, project, model_name, ix), 'GET',
+             None) for ix in range(len(versions) - 1)
+        ]
+
+        with _TestCloudMLHook(
+                self,
+                responses=responses,
+                expected_requests=expected_requests) as cml_hook:
+            list_versions_response = cml_hook.list_versions(
+                project_name=project, model_name=model_name)
+            self.assertEquals(list_versions_response, versions)
+
+    @_SKIP_IF
+    def test_delete_version(self):
+        project = 'test-project'
+        model_name = 'test-model'
+        version = 'test-version'
+        operation_name = 'projects/{}/operations/test-operation'.format(
+            project)
+
+        not_done_response_body = {'name': operation_name, 'done': False}
+        done_response_body = {'name': operation_name, 'done': True}
+        not_done_response = (
+            {'status': '200'}, json.dumps(not_done_response_body))
+        succeeded_response = (
+            {'status': '200'}, json.dumps(done_response_body))
+
+        expected_requests = [
+            ('{}projects/{}/models/{}/versions/{}?alt=json'.format(
+                self._SERVICE_URI_PREFIX, project, model_name, version), 'DELETE',
+             None),
+            ('{}{}?alt=json'.format(self._SERVICE_URI_PREFIX, operation_name),
+             'GET', None),
+        ]
+
+        with _TestCloudMLHook(
+                self,
+                responses=[not_done_response, succeeded_response],
+                expected_requests=expected_requests) as cml_hook:
+            delete_version_response = cml_hook.delete_version(
+                project_name=project, model_name=model_name, version_name=version)
+            self.assertEquals(delete_version_response, done_response_body)
+
+    @_SKIP_IF
+    def test_create_model(self):
+        project = 'test-project'
+        model_name = 'test-model'
+        model = {
+            'name': model_name,
+        }
+        response_body = {}
+        succeeded_response = ({'status': '200'}, json.dumps(response_body))
+
+        expected_requests = [
+            ('{}projects/{}/models?alt=json'.format(
+                self._SERVICE_URI_PREFIX, project), 'POST',
+             json.dumps(model)),
+        ]
+
+        with _TestCloudMLHook(
+                self,
+                responses=[succeeded_response],
+                expected_requests=expected_requests) as cml_hook:
+            create_model_response = cml_hook.create_model(
+                project_name=project, model=model)
+            self.assertEquals(create_model_response, response_body)
+
+    @_SKIP_IF
+    def test_get_model(self):
+        project = 'test-project'
+        model_name = 'test-model'
+        response_body = {'model': model_name}
+        succeeded_response = ({'status': '200'}, json.dumps(response_body))
+
+        expected_requests = [
+            ('{}projects/{}/models/{}?alt=json'.format(
+                self._SERVICE_URI_PREFIX, project, model_name), 'GET',
+             None),
+        ]
+
+        with _TestCloudMLHook(
+                self,
+                responses=[succeeded_response],
+                expected_requests=expected_requests) as cml_hook:
+            get_model_response = cml_hook.get_model(
+                project_name=project, model_name=model_name)
+            self.assertEquals(get_model_response, response_body)
+
+
+if __name__ == '__main__':
+    unittest.main()


[07/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/0809ee9d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/0809ee9d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/0809ee9d

Branch: refs/heads/master
Commit: 0809ee9db93936827a4417b9d7e018d3321e8b45
Parents: 33eb501 f5ffb9d
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Jul 12 10:08:37 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Jul 12 10:08:37 2017 -0700

----------------------------------------------------------------------
 README.md                                       |   2 +
 airflow/contrib/hooks/gcp_cloudml_hook.py       |  82 ++++-----
 airflow/contrib/hooks/spark_submit_hook.py      |   8 +-
 airflow/contrib/operators/cloudml_operator.py   | 148 ++++++++++++++-
 airflow/hooks/hive_hooks.py                     |  29 ++-
 ...dc7_add_max_tries_column_to_task_instance.py | 106 +++++++++++
 airflow/models.py                               |   3 +
 airflow/operators/docker_operator.py            |   8 +-
 airflow/operators/mssql_to_hive.py              |   7 +-
 airflow/operators/mysql_to_hive.py              |   7 +-
 airflow/operators/s3_to_hive_operator.py        |  10 +-
 airflow/task_runner/base_task_runner.py         |   2 +-
 airflow/utils/dag_processing.py                 |   5 +-
 docs/integration.rst                            |  41 +++++
 tests/contrib/hooks/test_gcp_cloudml_hook.py    | 111 +++++++++++-
 tests/contrib/hooks/test_spark_submit_hook.py   |   6 +-
 .../contrib/operators/test_cloudml_operator.py  | 179 ++++++++++++++-----
 .../operators/test_spark_submit_operator.py     |  31 ++--
 tests/jobs.py                                   |  21 ++-
 tests/operators/docker_operator.py              |   7 +-
 tests/operators/operators.py                    |  16 ++
 21 files changed, 696 insertions(+), 133 deletions(-)
----------------------------------------------------------------------



[05/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/e8b6988e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/e8b6988e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/e8b6988e

Branch: refs/heads/master
Commit: e8b6988ecd19d13cb7e6a106529d378752ca9be1
Parents: 572ca47 e92d6bf
Author: Chris Riccomini <cr...@apache.org>
Authored: Thu Jun 29 11:52:54 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Thu Jun 29 11:52:54 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/gcp_cloudml_hook.py       | 116 +++++-
 airflow/contrib/operators/cloudml_operator.py   | 356 +++++++++++++++----
 tests/contrib/hooks/test_gcp_cloudml_hook.py    |  79 +++-
 .../contrib/operators/test_cloudml_operator.py  | 288 +++++++++++++++
 4 files changed, 755 insertions(+), 84 deletions(-)
----------------------------------------------------------------------



[25/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/553744a7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/553744a7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/553744a7

Branch: refs/heads/master
Commit: 553744a7ebca4d47d881c90356fa5378a8aec605
Parents: 1075ea5 1e2d237
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Aug 23 13:26:13 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Aug 23 13:26:13 2017 -0700

----------------------------------------------------------------------
 README.md | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[06/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/33eb5015
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/33eb5015
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/33eb5015

Branch: refs/heads/master
Commit: 33eb5015681d8a314f82a6a1ece9975b72035db3
Parents: e8b6988 d231dce
Author: Chris Riccomini <cr...@apache.org>
Authored: Thu Jul 6 11:36:22 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Thu Jul 6 11:36:22 2017 -0700

----------------------------------------------------------------------
 airflow/config_templates/default_test.cfg       |   3 +
 airflow/contrib/hooks/__init__.py               |   3 +-
 airflow/contrib/hooks/gcp_pubsub_hook.py        |  93 ++++++++++
 airflow/contrib/operators/dataproc_operator.py  |   6 +-
 airflow/contrib/operators/pubsub_operator.py    | 170 +++++++++++++++++++
 airflow/executors/celery_executor.py            |  25 ++-
 airflow/utils/email.py                          |   1 +
 airflow/www/utils.py                            |  14 ++
 airflow/www/views.py                            |  20 +--
 tests/contrib/hooks/gcp_pubsub_hook.py          | 102 +++++++++++
 tests/contrib/operators/pubsub_operator.py      |  77 +++++++++
 .../contrib/operators/test_dataproc_operator.py |  12 +-
 tests/www/test_utils.py                         |  36 ++++
 13 files changed, 529 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[32/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/c781ff61
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/c781ff61
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/c781ff61

Branch: refs/heads/master
Commit: c781ff61cee1826a9b557758b43a44c9bc7155c8
Parents: a3ee59d 03af610
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 6 16:46:18 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 6 16:46:18 2017 -0700

----------------------------------------------------------------------
 airflow/bin/cli.py                      |  6 +-
 airflow/contrib/hooks/bigquery_hook.py  |  4 +-
 airflow/jobs.py                         | 47 +++++++-------
 airflow/models.py                       | 95 +++++++++++++++++++++++++---
 airflow/task_runner/base_task_runner.py |  3 -
 tests/dags/test_mark_success.py         | 30 +++++++++
 tests/jobs.py                           | 58 +++++++++++++++--
 tests/models.py                         | 18 +++++-
 8 files changed, 210 insertions(+), 51 deletions(-)
----------------------------------------------------------------------



[16/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/c08842f3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/c08842f3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/c08842f3

Branch: refs/heads/master
Commit: c08842f3ce82d5631d0d10a6407284d13b6901e5
Parents: c616eaa 1932ccc
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Jul 31 13:30:49 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Jul 31 13:30:49 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/operators/cloudml_operator.py   | 16 +++-
 .../contrib/operators/cloudml_operator_utils.py | 84 ++++++++++++--------
 airflow/contrib/operators/dataflow_operator.py  |  2 +
 .../operators/test_cloudml_operator_utils.py    | 40 +++++-----
 4 files changed, 91 insertions(+), 51 deletions(-)
----------------------------------------------------------------------



[29/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/2c95456c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/2c95456c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/2c95456c

Branch: refs/heads/master
Commit: 2c95456c8f8da9b8e0f6ea27fd254780d26f458a
Parents: b1e355c 4c674cc
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 6 09:45:42 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 6 09:45:42 2017 -0700

----------------------------------------------------------------------
 .../config_templates/default_airflow_logging.py |  2 +-
 airflow/contrib/hooks/bigquery_hook.py          |  6 +-
 airflow/contrib/hooks/spark_sql_hook.py         | 33 ++++-----
 airflow/contrib/operators/bigquery_operator.py  |  8 ++-
 airflow/contrib/operators/spark_sql_operator.py |  3 +
 airflow/utils/log/file_task_handler.py          | 25 +++++--
 airflow/utils/log/gcs_task_handler.py           |  8 +--
 airflow/utils/log/s3_task_handler.py            |  8 +--
 .../operators/test_spark_sql_operator.py        | 73 ++++++++++++++++++++
 tests/utils/test_log_handlers.py                | 23 ++++++
 10 files changed, 152 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[19/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/5ff7e838
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/5ff7e838
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/5ff7e838

Branch: refs/heads/master
Commit: 5ff7e83878417d72a877464d92bb5a57c212936f
Parents: fcf31d8 111ce57
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Aug 7 09:25:38 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Aug 7 09:25:38 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/operators/bigquery_check_operator.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[14/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/150d4726
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/150d4726
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/150d4726

Branch: refs/heads/master
Commit: 150d4726b36277fe6c198b3ed2344e77c4b84eb8
Parents: 016f48d 547f818
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Jul 31 13:21:18 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Jul 31 13:21:18 2017 -0700

----------------------------------------------------------------------
 airflow/jobs.py      | 515 ++++++++++++++++++++++++++++++----------------
 airflow/models.py    |  31 +++
 docs/integration.rst |  97 +++++----
 tests/jobs.py        | 189 ++++++++++++-----
 4 files changed, 563 insertions(+), 269 deletions(-)
----------------------------------------------------------------------



[02/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/b9809597
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/b9809597
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/b9809597

Branch: refs/heads/master
Commit: b98095979872d6905d1b8cfe7acc701472379d28
Parents: 265b293 e4b240f
Author: Chris Riccomini <cr...@apache.org>
Authored: Tue Jun 27 09:55:27 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Tue Jun 27 09:55:27 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/operators/dataproc_operator.py  |  6 +-
 .../contrib/operators/test_dataproc_operator.py | 89 ++++++++++++--------
 2 files changed, 58 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[03/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/971d08c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/971d08c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/971d08c0

Branch: refs/heads/master
Commit: 971d08c0498f245c45f21371fc5f6e98b1dfac65
Parents: b980959 5fe25d8
Author: Chris Riccomini <cr...@apache.org>
Authored: Tue Jun 27 12:42:46 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Tue Jun 27 12:42:46 2017 -0700

----------------------------------------------------------------------
 airflow/jobs.py   | 34 ++++++++++-----------
 airflow/models.py |  6 ++--
 tests/jobs.py     | 81 ++++++++++++++++++++++++++++++++++++++++++++++++++
 tests/models.py   |  8 +++++
 4 files changed, 108 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[04/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/572ca475
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/572ca475
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/572ca475

Branch: refs/heads/master
Commit: 572ca4759e2805cdd1a45517934d6bd61248ed11
Parents: 971d08c e414844
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Jun 28 15:23:18 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Jun 28 15:23:18 2017 -0700

----------------------------------------------------------------------
 airflow/bin/cli.py                             |  5 +----
 airflow/config_templates/default_airflow.cfg   |  7 -------
 airflow/config_templates/default_test.cfg      |  2 --
 airflow/contrib/operators/dataproc_operator.py | 22 +++++++++++++++++----
 4 files changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------



[26/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/b8b71ffc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/b8b71ffc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/b8b71ffc

Branch: refs/heads/master
Commit: b8b71ffc6c3ac2ee51a2dfe9a6af0b58ad63ed24
Parents: 553744a 4a4b024
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Aug 23 14:38:01 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Aug 23 14:38:01 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/bigquery_hook.py | 6 ++++++
 airflow/contrib/operators/gcs_to_bq.py | 5 +++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------



[21/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/0a2902c6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/0a2902c6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/0a2902c6

Branch: refs/heads/master
Commit: 0a2902c640637fc76072d1265b80763d06c633d2
Parents: 9660293 d295cf1
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Aug 14 11:31:19 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Aug 14 11:31:19 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/operators/file_to_gcs.py | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[27/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/b7cdc13f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/b7cdc13f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/b7cdc13f

Branch: refs/heads/master
Commit: b7cdc13fafefa5630587205b22c477200ee23777
Parents: b8b71ff 5c80755
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Aug 30 09:13:10 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Aug 30 09:13:10 2017 -0700

----------------------------------------------------------------------
 README.md | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------



[20/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/96602935
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/96602935
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/96602935

Branch: refs/heads/master
Commit: 966029357be5bcad2826cadb7333c1da9a619394
Parents: 5ff7e83 0d0cc62
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Aug 14 11:25:50 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Aug 14 11:25:50 2017 -0700

----------------------------------------------------------------------
 README.md                                       |  12 +-
 UPDATING.md                                     |  29 +--
 airflow/bin/cli.py                              | 106 ++-------
 airflow/config_templates/__init__.py            |  13 ++
 airflow/config_templates/default_airflow.cfg    |   7 +
 .../config_templates/default_airflow_logging.py |  94 ++++++++
 airflow/dag/__init__.py                         |   1 -
 ...dc7_add_max_tries_column_to_task_instance.py |   9 +-
 airflow/models.py                               |   9 +-
 airflow/settings.py                             |  14 ++
 airflow/task_runner/base_task_runner.py         |   2 +
 airflow/utils/log/__init__.py                   |  13 ++
 airflow/utils/log/file_task_handler.py          | 176 ++++++++++++++
 airflow/utils/log/gcs_task_handler.py           |  95 ++++++++
 airflow/utils/log/s3_task_handler.py            |  91 ++++++++
 airflow/utils/logging.py                        |  87 +++----
 airflow/www/views.py                            | 230 +++++++------------
 docs/configuration.rst                          |  15 +-
 tests/utils/test_log_handlers.py                |  73 ++++++
 tests/utils/test_logging.py                     |  96 +++++++-
 tests/www/test_views.py                         |  58 +++++
 21 files changed, 896 insertions(+), 334 deletions(-)
----------------------------------------------------------------------



[10/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/173860af
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/173860af
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/173860af

Branch: refs/heads/master
Commit: 173860afe938f91356dfdff7e039f3cbc5a16fb4
Parents: efa1de6 751e936
Author: Chris Riccomini <cr...@apache.org>
Authored: Thu Jul 20 10:40:06 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Thu Jul 20 10:40:06 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/spark_submit_hook.py      |  23 +-
 .../contrib/operators/cloudml_operator_utils.py | 223 +++++++++
 .../operators/cloudml_prediction_summary.py     | 177 +++++++
 airflow/jobs.py                                 | 334 +++++++++----
 tests/contrib/hooks/test_spark_submit_hook.py   |  26 +-
 .../operators/test_cloudml_operator_utils.py    | 179 +++++++
 tests/jobs.py                                   | 485 ++++++++++++++++++-
 7 files changed, 1317 insertions(+), 130 deletions(-)
----------------------------------------------------------------------



[23/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/71817093
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/71817093
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/71817093

Branch: refs/heads/master
Commit: 71817093db67dbf06ca458cb04e3e2bad4517dbf
Parents: 3c7ad18 de99aa2
Author: Chris Riccomini <cr...@apache.org>
Authored: Fri Aug 18 14:03:45 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Fri Aug 18 14:03:45 2017 -0700

----------------------------------------------------------------------
 UPDATING.md                                     |   6 +
 airflow/config_templates/default_airflow.cfg    |   7 +
 airflow/config_templates/default_test.cfg       |   2 +
 airflow/contrib/operators/dataproc_operator.py  |   2 +
 airflow/contrib/operators/druid_operator.py     |  47 ++++
 airflow/contrib/operators/ssh_operator.py       |  10 +-
 airflow/hooks/druid_hook.py                     | 215 +++++--------------
 .../947454bf1dff_add_ti_job_id_index.py         |  38 ++++
 ...c56_make_xcom_value_column_a_large_binary.py |  45 ++++
 airflow/models.py                               | 156 ++++++++++----
 airflow/operators/hive_to_druid.py              | 116 ++++++++--
 airflow/ti_deps/deps/trigger_rule_dep.py        |   4 +-
 airflow/utils/db.py                             |   4 +
 airflow/www/views.py                            |   9 +-
 docs/concepts.rst                               |   3 +-
 scripts/ci/requirements.txt                     |   2 +-
 setup.py                                        |   5 +-
 .../contrib/operators/test_dataproc_operator.py | 109 ++++++++++
 tests/contrib/operators/test_druid_operator.py  |  47 ++++
 tests/contrib/operators/test_sftp_operator.py   |  84 +++++++-
 tests/contrib/operators/test_ssh_operator.py    |  24 ++-
 tests/hooks/test_druid_hook.py                  |  99 +++++++++
 tests/models.py                                 | 124 ++++++++++-
 tests/operators/test_hive_to_druid.py           | 132 ++++++++++++
 24 files changed, 1057 insertions(+), 233 deletions(-)
----------------------------------------------------------------------



[15/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/c616eaa2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/c616eaa2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/c616eaa2

Branch: refs/heads/master
Commit: c616eaa2e27ed315bf66993c57451e3a5e2f7076
Parents: 150d472 6e26407
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Jul 31 13:24:40 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Jul 31 13:24:40 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/bigquery_hook.py         |  7 ++++++-
 airflow/contrib/operators/bigquery_operator.py | 13 +++++++++++--
 2 files changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[11/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/f1f0d23e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/f1f0d23e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/f1f0d23e

Branch: refs/heads/master
Commit: f1f0d23e5d17dc77e7477cc8a327d9fd09891fed
Parents: 173860a b49986c
Author: Chris Riccomini <cr...@apache.org>
Authored: Fri Jul 21 13:04:32 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Fri Jul 21 13:04:32 2017 -0700

----------------------------------------------------------------------
 UPDATING.md                                     |   9 +-
 airflow/bin/cli.py                              |  81 +++---
 airflow/contrib/hooks/ssh_hook.py               | 268 +++++++++++--------
 airflow/contrib/operators/__init__.py           |   2 +-
 airflow/contrib/operators/sftp_operator.py      |  99 +++++++
 .../contrib/operators/ssh_execute_operator.py   | 159 -----------
 airflow/contrib/operators/ssh_operator.py       | 106 ++++++++
 airflow/models.py                               |  50 ++--
 airflow/utils/logging.py                        |  66 +++++
 airflow/www/templates/airflow/ti_log.html       |  40 +++
 airflow/www/views.py                            | 180 +++++++------
 dags/test_dag.py                                |   4 +-
 docs/code.rst                                   |   2 +-
 docs/scheduler.rst                              |  19 +-
 scripts/ci/requirements.txt                     |   1 +
 setup.py                                        |   7 +-
 tests/contrib/hooks/test_ssh_hook.py            |  70 +++++
 tests/contrib/operators/test_sftp_operator.py   | 158 +++++++++++
 .../operators/test_ssh_execute_operator.py      |  95 -------
 tests/contrib/operators/test_ssh_operator.py    |  89 ++++++
 tests/core.py                                   |  52 ----
 tests/models.py                                 | 137 +++++++++-
 tests/operators/python_operator.py              |  88 +++---
 tests/utils/test_dates.py                       |   4 -
 tests/utils/test_logging.py                     |  29 ++
 25 files changed, 1198 insertions(+), 617 deletions(-)
----------------------------------------------------------------------



[24/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/1075ea56
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/1075ea56
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/1075ea56

Branch: refs/heads/master
Commit: 1075ea567019a56e4b124a6eb170af1cd67c0dfe
Parents: 7181709 f1a7c00
Author: Chris Riccomini <cr...@apache.org>
Authored: Fri Aug 18 15:30:58 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Fri Aug 18 15:30:58 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/operators/bigquery_table_delete_operator.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[34/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/004a3477
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/004a3477
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/004a3477

Branch: refs/heads/master
Commit: 004a347727fa1ccad4598e8241cbd473048616fd
Parents: e822425 ea9ab96
Author: Chris Riccomini <cr...@apache.org>
Authored: Fri Sep 8 11:23:06 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Fri Sep 8 11:23:06 2017 -0700

----------------------------------------------------------------------
 airflow/operators/check_operator.py |  4 ++--
 docs/integration.rst                | 38 ++++++++++++++++----------------
 2 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[37/37] incubator-airflow git commit: [AIRFLOW-XXX] Bumping Airflow 1.10.0dev0+incubating version

Posted by cr...@apache.org.
[AIRFLOW-XXX] Bumping Airflow 1.10.0dev0+incubating version


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/a8d486de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/a8d486de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/a8d486de

Branch: refs/heads/master
Commit: a8d486de337779e6319b90215fd8ea7faa5f5913
Parents: 78124a2
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 13 10:39:26 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 13 10:39:26 2017 -0700

----------------------------------------------------------------------
 airflow/version.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a8d486de/airflow/version.py
----------------------------------------------------------------------
diff --git a/airflow/version.py b/airflow/version.py
index bce497e..55e8b9f 100644
--- a/airflow/version.py
+++ b/airflow/version.py
@@ -13,4 +13,4 @@
 # limitations under the License.
 #
 
-version = '1.9.0dev0+incubating'
+version = '1.10.0dev0+incubating'


[35/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/9bc0d399
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/9bc0d399
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/9bc0d399

Branch: refs/heads/master
Commit: 9bc0d399a6d303ff1068b880f1de76dfd5846c6f
Parents: 004a347 6ac2963
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 13 09:31:44 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 13 09:31:44 2017 -0700

----------------------------------------------------------------------
 .rat-excludes                                   |  10 +
 CHANGELOG.txt                                   |  45 +++
 INSTALL                                         |   9 +
 README.md                                       |   1 +
 UPDATING.md                                     |   4 +-
 airflow/__init__.py                             |  10 +-
 airflow/api/__init__.py                         |  12 +-
 airflow/api/auth/backend/kerberos_auth.py       |  14 +-
 airflow/api/common/experimental/get_task.py     |   4 -
 .../common/experimental/get_task_instance.py    |   4 -
 airflow/bin/airflow                             |   1 -
 airflow/bin/cli.py                              |  46 +--
 airflow/configuration.py                        |  25 +-
 .../auth/backends/github_enterprise_auth.py     |   5 +-
 airflow/contrib/auth/backends/google_auth.py    |  11 +-
 airflow/contrib/auth/backends/kerberos_auth.py  |   5 +-
 airflow/contrib/auth/backends/ldap_auth.py      |  32 +-
 airflow/contrib/auth/backends/password_auth.py  |  10 +-
 airflow/contrib/executors/mesos_executor.py     |  51 +--
 airflow/contrib/hooks/bigquery_hook.py          |  66 ++--
 airflow/contrib/hooks/cloudant_hook.py          |  10 +-
 airflow/contrib/hooks/databricks_hook.py        |  32 +-
 airflow/contrib/hooks/datadog_hook.py           |  13 +-
 airflow/contrib/hooks/datastore_hook.py         |   3 +-
 airflow/contrib/hooks/ftp_hook.py               |  10 +-
 airflow/contrib/hooks/gcp_api_base_hook.py      |  13 +-
 airflow/contrib/hooks/gcp_dataflow_hook.py      |  29 +-
 airflow/contrib/hooks/gcp_dataproc_hook.py      |  28 +-
 airflow/contrib/hooks/gcp_mlengine_hook.py      |  48 ++-
 airflow/contrib/hooks/gcs_hook.py               |  10 +-
 airflow/contrib/hooks/jira_hook.py              |   9 +-
 airflow/contrib/hooks/qubole_hook.py            |  22 +-
 airflow/contrib/hooks/redis_hook.py             |  16 +-
 airflow/contrib/hooks/salesforce_hook.py        |  31 +-
 airflow/contrib/hooks/spark_sql_hook.py         |  10 +-
 airflow/contrib/hooks/spark_submit_hook.py      |  24 +-
 airflow/contrib/hooks/sqoop_hook.py             |  17 +-
 airflow/contrib/hooks/ssh_hook.py               |  34 +-
 airflow/contrib/hooks/wasb_hook.py              |  18 +
 airflow/contrib/operators/bigquery_operator.py  |   4 +-
 .../operators/bigquery_table_delete_operator.py |   4 +-
 .../contrib/operators/bigquery_to_bigquery.py   |   8 +-
 airflow/contrib/operators/bigquery_to_gcs.py    |   4 +-
 .../contrib/operators/databricks_operator.py    |  29 +-
 airflow/contrib/operators/dataproc_operator.py  |  30 +-
 .../operators/datastore_export_operator.py      |   5 +-
 .../operators/datastore_import_operator.py      |   6 +-
 airflow/contrib/operators/ecs_operator.py       |  24 +-
 .../contrib/operators/emr_add_steps_operator.py |   7 +-
 .../operators/emr_create_job_flow_operator.py   |  10 +-
 .../emr_terminate_job_flow_operator.py          |   7 +-
 airflow/contrib/operators/file_to_wasb.py       |  14 +-
 airflow/contrib/operators/fs_operator.py        |   4 +-
 .../contrib/operators/gcs_download_operator.py  |   6 +-
 airflow/contrib/operators/gcs_to_bq.py          |  18 +-
 airflow/contrib/operators/hipchat_operator.py   |   3 +-
 airflow/contrib/operators/mlengine_operator.py  |  30 +-
 .../operators/mlengine_prediction_summary.py    |   2 -
 airflow/contrib/operators/mysql_to_gcs.py       |   5 +-
 airflow/contrib/operators/sftp_operator.py      |   7 +-
 .../contrib/operators/spark_submit_operator.py  |   6 +-
 airflow/contrib/operators/ssh_operator.py       |   1 -
 airflow/contrib/operators/vertica_operator.py   |   5 +-
 airflow/contrib/operators/vertica_to_hive.py    |   5 +-
 airflow/contrib/sensors/bigquery_sensor.py      |   5 +-
 airflow/contrib/sensors/datadog_sensor.py       |   5 +-
 airflow/contrib/sensors/emr_base_sensor.py      |   9 +-
 airflow/contrib/sensors/emr_job_flow_sensor.py  |   6 +-
 airflow/contrib/sensors/emr_step_sensor.py      |   5 +-
 airflow/contrib/sensors/ftp_sensor.py           |   4 +-
 airflow/contrib/sensors/gcs_sensor.py           |   7 +-
 airflow/contrib/sensors/hdfs_sensors.py         |  12 +-
 airflow/contrib/sensors/jira_sensor.py          |  27 +-
 airflow/contrib/sensors/redis_key_sensor.py     |   4 -
 airflow/contrib/sensors/wasb_sensor.py          |  11 +-
 .../contrib/task_runner/cgroup_task_runner.py   |  49 +--
 airflow/executors/__init__.py                   |   8 +-
 airflow/executors/base_executor.py              |  18 +-
 airflow/executors/celery_executor.py            |  24 +-
 airflow/executors/dask_executor.py              |  10 +-
 airflow/executors/local_executor.py             |  11 +-
 airflow/executors/sequential_executor.py        |   4 +-
 airflow/hooks/S3_hook.py                        |  52 +--
 airflow/hooks/base_hook.py                      |   9 +-
 airflow/hooks/dbapi_hook.py                     |  33 +-
 airflow/hooks/druid_hook.py                     |  10 +-
 airflow/hooks/hive_hooks.py                     |  37 +-
 airflow/hooks/http_hook.py                      |   7 +-
 airflow/hooks/oracle_hook.py                    |   9 +-
 airflow/hooks/pig_hook.py                       |   6 +-
 airflow/hooks/presto_hook.py                    |   4 -
 airflow/hooks/webhdfs_hook.py                   |  17 +-
 airflow/hooks/zendesk_hook.py                   |  20 +-
 airflow/jobs.py                                 | 352 +++++++++----------
 ...d2ae31099d61_increase_text_size_for_mysql.py |  41 +++
 airflow/models.py                               | 231 ++++++------
 airflow/operators/bash_operator.py              |  23 +-
 airflow/operators/check_operator.py             |  24 +-
 airflow/operators/dagrun_operator.py            |   6 +-
 airflow/operators/docker_operator.py            |  12 +-
 airflow/operators/generic_transfer.py           |  13 +-
 airflow/operators/hive_operator.py              |   4 +-
 airflow/operators/hive_stats_operator.py        |  10 +-
 airflow/operators/hive_to_druid.py              |  18 +-
 airflow/operators/hive_to_mysql.py              |  14 +-
 airflow/operators/hive_to_samba_operator.py     |   6 +-
 airflow/operators/http_operator.py              |   8 +-
 airflow/operators/jdbc_operator.py              |   7 +-
 airflow/operators/latest_only_operator.py       |  19 +-
 airflow/operators/mssql_operator.py             |   5 +-
 airflow/operators/mssql_to_hive.py              |   6 +-
 airflow/operators/mysql_operator.py             |   5 +-
 airflow/operators/mysql_to_hive.py              |   5 +-
 airflow/operators/oracle_operator.py            |   5 +-
 airflow/operators/pig_operator.py               |   4 +-
 airflow/operators/postgres_operator.py          |   5 +-
 airflow/operators/presto_to_mysql.py            |  12 +-
 airflow/operators/python_operator.py            |  24 +-
 airflow/operators/redshift_to_s3_operator.py    |  17 +-
 airflow/operators/s3_file_transform_operator.py |  30 +-
 airflow/operators/s3_to_hive_operator.py        |  39 +-
 airflow/operators/sensors.py                    |  49 +--
 airflow/operators/slack_operator.py             |   9 +-
 airflow/operators/sqlite_operator.py            |   5 +-
 airflow/plugins_manager.py                      |  11 +-
 airflow/security/kerberos.py                    |  25 +-
 airflow/settings.py                             |  17 +-
 airflow/task_runner/base_task_runner.py         |   9 +-
 airflow/utils/dag_processing.py                 |  55 +--
 airflow/utils/db.py                             |  15 +-
 airflow/utils/email.py                          |   8 +-
 airflow/utils/log/LoggingMixin.py               |  45 +++
 airflow/utils/log/file_task_handler.py          |  34 +-
 airflow/utils/log/gcs_task_handler.py           | 125 ++++++-
 airflow/utils/log/s3_task_handler.py            |  97 ++++-
 airflow/utils/logging.py                        | 252 -------------
 airflow/utils/timeout.py                        |  17 +-
 airflow/www/api/experimental/endpoints.py       |   6 +-
 airflow/www/app.py                              |  10 +-
 airflow/www/views.py                            |  23 +-
 dev/airflow-pr                                  |  11 +-
 docs/integration.rst                            |  96 ++++-
 setup.py                                        |  15 +-
 tests/contrib/hooks/test_databricks_hook.py     |  52 ++-
 tests/contrib/hooks/test_wasb_hook.py           |  14 +-
 .../contrib/operators/test_dataproc_operator.py |  59 ++--
 tests/core.py                                   |  20 +-
 tests/operators/sensors.py                      |  53 +--
 tests/utils/log/test_logging.py                 | 108 ++++++
 tests/utils/test_logging.py                     | 103 ------
 150 files changed, 1906 insertions(+), 1791 deletions(-)
----------------------------------------------------------------------



[22/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/3c7ad18d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/3c7ad18d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/3c7ad18d

Branch: refs/heads/master
Commit: 3c7ad18d01f4b8a4a0eddb5944707efb2351240b
Parents: 0a2902c 42cad60
Author: Chris Riccomini <cr...@apache.org>
Authored: Tue Aug 15 10:21:48 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Tue Aug 15 10:21:48 2017 -0700

----------------------------------------------------------------------
 airflow/task_runner/base_task_runner.py | 1 +
 airflow/www/views.py                    | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[18/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/fcf31d88
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/fcf31d88
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/fcf31d88

Branch: refs/heads/master
Commit: fcf31d8813b66f09fc6725959bfa0cdad9efc847
Parents: ccf3ccc ddc5026
Author: Chris Riccomini <cr...@apache.org>
Authored: Mon Aug 7 09:24:34 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Mon Aug 7 09:24:34 2017 -0700

----------------------------------------------------------------------
 airflow/bin/cli.py                        |  51 +++++++--
 airflow/jobs.py                           | 144 +++++++++++++++----------
 airflow/models.py                         | 119 ++++++++++++++++++--
 airflow/www/api/experimental/endpoints.py |   2 +-
 airflow/www/views.py                      |  17 ++-
 tests/core.py                             |  10 ++
 tests/jobs.py                             | 140 +++++++++++++++++++++---
 tests/models.py                           |  63 +++++++++++
 tests/www/test_views.py                   |  62 +++++++++++
 9 files changed, 511 insertions(+), 97 deletions(-)
----------------------------------------------------------------------



[17/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/ccf3ccc1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/ccf3ccc1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/ccf3ccc1

Branch: refs/heads/master
Commit: ccf3ccc10d36ca4b526fcf35bd365d0a37704042
Parents: c08842f 836f289
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Aug 2 12:52:58 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Aug 2 12:52:58 2017 -0700

----------------------------------------------------------------------
 airflow/operators/sensors.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[08/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/c0863e38
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/c0863e38
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/c0863e38

Branch: refs/heads/master
Commit: c0863e38dfa8aabfd3a1c76d2ee6c5354e528fde
Parents: 0809ee9 38c86bb
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Jul 12 10:54:57 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Jul 12 10:54:57 2017 -0700

----------------------------------------------------------------------
 airflow/configuration.py | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[31/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/a3ee59d7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/a3ee59d7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/a3ee59d7

Branch: refs/heads/master
Commit: a3ee59d7f3e19238d408e01020879c54bc102a5c
Parents: 2ae6a55 af91e2a
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 6 09:51:45 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 6 09:51:45 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/gcp_cloudml_hook.py       | 269 ---------
 airflow/contrib/hooks/gcp_mlengine_hook.py      | 269 +++++++++
 airflow/contrib/operators/cloudml_operator.py   | 565 -------------------
 .../contrib/operators/cloudml_operator_utils.py | 245 --------
 .../operators/cloudml_prediction_summary.py     | 177 ------
 airflow/contrib/operators/mlengine_operator.py  | 564 ++++++++++++++++++
 .../operators/mlengine_operator_utils.py        | 245 ++++++++
 .../operators/mlengine_prediction_summary.py    | 177 ++++++
 tests/contrib/hooks/test_gcp_cloudml_hook.py    | 413 --------------
 tests/contrib/hooks/test_gcp_mlengine_hook.py   | 413 ++++++++++++++
 .../contrib/operators/test_cloudml_operator.py  | 373 ------------
 .../operators/test_cloudml_operator_utils.py    | 183 ------
 .../contrib/operators/test_mlengine_operator.py | 373 ++++++++++++
 .../operators/test_mlengine_operator_utils.py   | 183 ++++++
 14 files changed, 2224 insertions(+), 2225 deletions(-)
----------------------------------------------------------------------



[36/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/78124a2a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/78124a2a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/78124a2a

Branch: refs/heads/master
Commit: 78124a2a6b31b6bbd33adce763ad3235245fd601
Parents: 9bc0d39 653562e
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 13 09:32:30 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 13 09:32:30 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/gcp_dataflow_hook.py | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------



[13/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/016f48d8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/016f48d8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/016f48d8

Branch: refs/heads/master
Commit: 016f48d85b4e19fe89be84150d0a127c8202fcbf
Parents: eca3234 426b6a6
Author: Chris Riccomini <cr...@apache.org>
Authored: Thu Jul 27 09:15:38 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Thu Jul 27 09:15:38 2017 -0700

----------------------------------------------------------------------
 airflow/config_templates/default_airflow.cfg   |   5 +
 airflow/config_templates/default_test.cfg      |   1 +
 airflow/contrib/hooks/bigquery_hook.py         |   8 +-
 airflow/contrib/operators/bigquery_operator.py |   8 +-
 airflow/jobs.py                                |  45 ++++++--
 airflow/models.py                              |   2 +-
 airflow/operators/sensors.py                   |  12 ++-
 tests/jobs.py                                  |  38 +++++++
 tests/operators/sensors.py                     | 110 +++++++++++++++++++-
 9 files changed, 210 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[28/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/b1e355c3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/b1e355c3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/b1e355c3

Branch: refs/heads/master
Commit: b1e355c39b8cd4640e2e4060986a0c54424b8128
Parents: b7cdc13 de59321
Author: Chris Riccomini <cr...@apache.org>
Authored: Fri Sep 1 12:58:39 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Fri Sep 1 12:58:39 2017 -0700

----------------------------------------------------------------------
 README.md                                         |  3 ++-
 airflow/contrib/operators/dataproc_operator.py    | 14 ++++++++++++++
 airflow/operators/slack_operator.py               |  2 +-
 tests/contrib/operators/test_dataproc_operator.py | 10 +++++++++-
 4 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[12/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/eca32343
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/eca32343
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/eca32343

Branch: refs/heads/master
Commit: eca323432b9ada9da34d9499f2111bfb0f00e2a8
Parents: f1f0d23 b87903d
Author: Chris Riccomini <cr...@apache.org>
Authored: Fri Jul 21 13:06:57 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Fri Jul 21 13:06:57 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/operators/bigquery_table_delete_operator.py | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[30/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/2ae6a55d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/2ae6a55d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/2ae6a55d

Branch: refs/heads/master
Commit: 2ae6a55dcc4b5aa6361930a81c1bc40154ceb65f
Parents: 2c95456 86063ba
Author: Chris Riccomini <cr...@apache.org>
Authored: Wed Sep 6 09:50:47 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Wed Sep 6 09:50:47 2017 -0700

----------------------------------------------------------------------
 airflow/contrib/hooks/bigquery_hook.py          |  35 +++++--
 airflow/contrib/hooks/datastore_hook.py         | 100 +++++++++++++++---
 airflow/contrib/hooks/gcp_api_base_hook.py      |  20 +++-
 .../operators/datastore_export_operator.py      | 104 +++++++++++++++++++
 .../operators/datastore_import_operator.py      |  95 +++++++++++++++++
 airflow/contrib/operators/gcs_to_bq.py          |  12 ++-
 6 files changed, 335 insertions(+), 31 deletions(-)
----------------------------------------------------------------------



[09/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/efa1de6d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/efa1de6d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/efa1de6d

Branch: refs/heads/master
Commit: efa1de6dbd3e37ec7d9c935e6ef1565d2d1999cb
Parents: c0863e3 e88ecff
Author: Chris Riccomini <cr...@apache.org>
Authored: Thu Jul 13 17:05:31 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Thu Jul 13 17:05:31 2017 -0700

----------------------------------------------------------------------
 README.md                                       |  2 +
 airflow/bin/cli.py                              |  2 +-
 airflow/contrib/hooks/bigquery_hook.py          |  6 +++
 airflow/contrib/hooks/gcp_cloudml_hook.py       | 44 +++++++++---------
 airflow/contrib/operators/cloudml_operator.py   | 47 ++++++++++----------
 airflow/contrib/operators/gcs_to_bq.py          |  5 +++
 tests/contrib/hooks/test_gcp_cloudml_hook.py    | 20 ++++-----
 .../contrib/operators/test_cloudml_operator.py  |  2 +-
 8 files changed, 71 insertions(+), 57 deletions(-)
----------------------------------------------------------------------



[33/37] incubator-airflow git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow

Posted by cr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-airflow


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/e8224255
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/e8224255
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/e8224255

Branch: refs/heads/master
Commit: e822425575f047425457f964bf29c4bc1aa3e650
Parents: c781ff6 2d40694
Author: Chris Riccomini <cr...@apache.org>
Authored: Thu Sep 7 14:03:41 2017 -0700
Committer: Chris Riccomini <cr...@apache.org>
Committed: Thu Sep 7 14:03:41 2017 -0700

----------------------------------------------------------------------
 README.md                           | 2 ++
 airflow/operators/email_operator.py | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------