From 844eb2c8d0806d7420e948e73c93e0fc932234e6 Mon Sep 17 00:00:00 2001 From: Chris Riccomini Date: Thu, 28 Apr 2016 08:32:48 -0700 Subject: [PATCH] AIRFLOW-15: Remove gcloud --- airflow/contrib/hooks/__init__.py | 16 +- airflow/contrib/hooks/bigquery_hook.py | 15 ++ airflow/contrib/hooks/datastore_hook.py | 15 ++ airflow/contrib/hooks/ftp_hook.py | 15 ++ airflow/contrib/hooks/gc_base_hook.py | 15 ++ airflow/contrib/hooks/gcloud/base_hook.py | 136 ---------------- airflow/contrib/hooks/gcloud/gcs_hook.py | 184 ---------------------- airflow/contrib/hooks/gcloud/readme.md | 20 --- airflow/contrib/hooks/gcs_hook.py | 15 ++ airflow/contrib/hooks/qubole_hook.py | 15 ++ airflow/contrib/hooks/vertica_hook.py | 15 ++ airflow/utils/logging.py | 77 +++------ docs/installation.rst | 4 - setup.py | 4 - 14 files changed, 146 insertions(+), 400 deletions(-) delete mode 100644 airflow/contrib/hooks/gcloud/base_hook.py delete mode 100644 airflow/contrib/hooks/gcloud/gcs_hook.py delete mode 100644 airflow/contrib/hooks/gcloud/readme.md diff --git a/airflow/contrib/hooks/__init__.py b/airflow/contrib/hooks/__init__.py index b10b25d91d453..b4627f56f5014 100644 --- a/airflow/contrib/hooks/__init__.py +++ b/airflow/contrib/hooks/__init__.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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. +# + # Imports the hooks dynamically while keeping the package API clean, # abstracting the underlying modules from airflow.utils.helpers import import_module_attrs as _import_module_attrs @@ -5,7 +20,6 @@ _hooks = { 'ftp_hook': ['FTPHook'], 'ftps_hook': ['FTPSHook'], - 'gcloud/gcs_hook': ['GCSHook'], 'vertica_hook': ['VerticaHook'], 'ssh_hook': ['SSHHook'], 'bigquery_hook': ['BigQueryHook'], diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 592a2614dcf11..39bf6f0701850 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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. +# + """ This module contains a BigQuery Hook, as well as a very basic PEP 249 implementation for BigQuery. diff --git a/airflow/contrib/hooks/datastore_hook.py b/airflow/contrib/hooks/datastore_hook.py index 3b8a8eb20fb31..612f47c02bcca 100644 --- a/airflow/contrib/hooks/datastore_hook.py +++ b/airflow/contrib/hooks/datastore_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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. +# + from apiclient.discovery import build from airflow.contrib.hooks.gc_base_hook import GoogleCloudBaseHook diff --git a/airflow/contrib/hooks/ftp_hook.py b/airflow/contrib/hooks/ftp_hook.py index 841b959265645..5d5cc1132dc77 100644 --- a/airflow/contrib/hooks/ftp_hook.py +++ b/airflow/contrib/hooks/ftp_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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 datetime import ftplib import logging diff --git a/airflow/contrib/hooks/gc_base_hook.py b/airflow/contrib/hooks/gc_base_hook.py index 6af01e79eeffb..f6928fef85f8f 100644 --- a/airflow/contrib/hooks/gc_base_hook.py +++ b/airflow/contrib/hooks/gc_base_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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 httplib2 import logging diff --git a/airflow/contrib/hooks/gcloud/base_hook.py b/airflow/contrib/hooks/gcloud/base_hook.py deleted file mode 100644 index 5c69b4b0a0869..0000000000000 --- a/airflow/contrib/hooks/gcloud/base_hook.py +++ /dev/null @@ -1,136 +0,0 @@ -# -*- coding: utf-8 -*- -# -# 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. - -from airflow.hooks import BaseHook -import gcloud - -class GCPBaseHook(BaseHook): - """ - A hook for working wth Google Cloud Platform via the gcloud library. - - A GCP connection ID can be provided. If it is provided, its values - will OVERRIDE any argments passed to the hook. The following precendance is - observed: - GCP connection fields - GCPBaseHook initialization arguments - host environment - - Google Cloud Platform connections can be created from the Airflow UI. If - created manually, the relevant (but optional) fields should be added to - the connection's "extra" field as JSON: - { - "project": "", - "key_path": "" - "service_account": "" - "scope": "" - } - - service_account is only required if the key_path points to a P12 file. - - scope is only used if key_path is provided. Scopes can include, for example: - https://www.googleapis.com/auth/devstorage.full_control - https://www.googleapis.com/auth/devstorage.read_only - https://www.googleapis.com/auth/devstorage.read_write - - If fields are not provided, either as arguments or extras, they can be set - in the host environment. - - To set a default project, use: - gcloud config set project - - To log in: - gcloud auth - - """ - - client_class = None - - def __init__( - self, - gcp_conn_id=None, - project=None, - key_path=None, - service_account=None, - scope=None, - *args, - **kwargs): - - if not self.client_class: - raise NotImplementedError( - 'The GCPBaseHook must be extended by providing a client_class.') - - # compatibility with GoogleCloudStorageHook - if 'google_cloud_storage_conn_id' in kwargs and not gcp_conn_id: - gcp_conn_id = kwargs.pop('google_cloud_storage_conn_id') - - self.gcp_conn_id = gcp_conn_id - self.project = project - self.key_path = key_path - self.service_account = service_account - self.scope = scope - - self.client = self.get_conn() - - def get_conn(self): - # parse arguments and connection extras - if self.gcp_conn_id: - extras = self.get_connection(self.gcp_conn_id).extra_dejson - else: - extras = {} - - def load_field(f, fallback=None): - # long_f: the format for UI-created fields - long_f = 'extra__google_cloud_platform__{}'.format(f) - if long_f in extras: - return extras[long_f] - elif f in extras: - return extras[f] - else: - return getattr(self, fallback or f) - - project = load_field('project') - key_path = load_field('key_path') - service_account = load_field('service_account') - scope = load_field('scope') - if scope: - scope = scope.split(',') - - # guess project, if possible - if not project: - project = gcloud._helpers._determine_default_project() - # workaround for - # https://github.com/GoogleCloudPlatform/gcloud-python/issues/1470 - if isinstance(project, bytes): - project = project.decode() - - # load credentials/scope - if key_path: - if key_path.endswith('.json') or key_path.endswith('.JSON'): - credentials = gcloud.credentials.get_for_service_account_json( - json_credentials_path=key_path, - scope=scope) - elif key_path.endswith('.p12') or key_path.endswith('.P12'): - credentials = gcloud.credentials.get_for_service_account_p12( - client_email=service_account, - private_key_path=key_path, - scope=scope) - else: - raise ValueError('Unrecognized keyfile: {}'.format(key_path)) - client = self.client_class( - credentials=credentials, - project=project) - else: - client = self.client_class(project=project) - - return client diff --git a/airflow/contrib/hooks/gcloud/gcs_hook.py b/airflow/contrib/hooks/gcloud/gcs_hook.py deleted file mode 100644 index f762a0a565c5b..0000000000000 --- a/airflow/contrib/hooks/gcloud/gcs_hook.py +++ /dev/null @@ -1,184 +0,0 @@ -# -*- coding: utf-8 -*- -# -# 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. - -from future.standard_library import install_aliases -install_aliases() - -from airflow.contrib.hooks.gcloud.base_hook import GCPBaseHook - -from urllib.parse import urlparse -from airflow.utils import AirflowException - -import gcloud.storage - -def parse_gcs_url(gsurl): - """ - Given a Google Cloud Storage URL (gs:///), returns a - tuple containing the corresponding bucket and blob. - """ - parsed_url = urlparse(gsurl) - if not parsed_url.netloc: - raise AirflowException('Please provide a bucket name') - else: - bucket = parsed_url.netloc - blob = parsed_url.path.strip('/') - return (bucket, blob) - -class GCSHook(GCPBaseHook): - - client_class = gcloud.storage.Client - - def bucket_exists(self, bucket): - return self.client.bucket(bucket).exists() - - def get_bucket(self, bucket): - return self.client.get_bucket(bucket) - - def list_blobs( - self, - bucket, - max_results=None, - page_token=None, - prefix=None, - delimiter=None): - return self.client.bucket(bucket).list_blobs( - max_results=max_results, - page_token=page_token, - prefix=prefix, - delimiter=delimiter) - - def get_blob(self, blob, bucket=None): - """ - Returns None if the blob does not exist - """ - if not bucket: - bucket, blob = parse_gcs_url(blob) - return self.client.bucket(bucket).get_blob(blob) - - def blob_exists(self, blob, bucket=None): - if not bucket: - bucket, blob = parse_gcs_url(blob) - return self.client.bucket(bucket).blob(blob).exists() - - def upload_from_file( - self, - file_obj, - blob, - bucket=None, - replace=False): - if not bucket: - bucket, blob = parse_gcs_url(blob) - gcs_blob = self.client.bucket(bucket).blob(blob) - if gcs_blob.exists() and not replace: - raise ValueError( - 'The blob {bucket}/{blob} already exists.'.format(**locals())) - gcs_blob.upload_from_file(file_obj) - - def upload_from_filename( - self, - filename, - blob, - bucket=None, - replace=False): - if not bucket: - bucket, blob = parse_gcs_url(blob) - gcs_blob = self.client.bucket(bucket).blob(blob) - if gcs_blob.exists() and not replace: - raise ValueError( - 'The blob {bucket}/{blob} already exists.'.format(**locals())) - gcs_blob.upload_from_filename(filename) - - def upload_from_string( - self, - string, - blob, - bucket=None, - replace=False): - if not bucket: - bucket, blob = parse_gcs_url(blob) - gcs_blob = self.client.bucket(bucket).blob(blob) - if gcs_blob.exists() and not replace: - raise ValueError( - 'The blob {bucket}/{blob} already exists.'.format(**locals())) - gcs_blob.upload_from_string(string) - - def download_as_string( - self, - blob, - bucket=None): - if not bucket: - bucket, blob = parse_gcs_url(blob) - gcs_blob = self.client.bucket(bucket).get_blob(blob) - if not gcs_blob: - raise ValueError( - 'Blob does not exist: {bucket}/{blob}'.format(**locals())) - return gcs_blob.download_as_string() - - def download_to_file( - self, - file_obj, - blob, - bucket=None): - if not bucket: - bucket, blob = parse_gcs_url(blob) - gcs_blob = self.client.bucket(bucket).get_blob(blob) - if not gcs_blob: - raise ValueError( - 'Blob does not exist: {bucket}/{blob}'.format(**locals())) - return gcs_blob.download_to_file(file_obj) - - def download_to_filename( - self, - filename, - blob, - bucket=None): - if not bucket: - bucket, blob = parse_gcs_url(blob) - gcs_blob = self.client.bucket(bucket).get_blob(blob) - if not gcs_blob: - raise ValueError( - 'Blob does not exist: {bucket}/{blob}'.format(**locals())) - return gcs_blob.download_to_filename(filename) - - # Compatibility methods - - def download( - self, - bucket, - object, - filename=False): - """ - This method is provided for compatibility with - contrib/GoogleCloudStorageHook. - """ - if filename: - return self.download_to_filename( - filename=filename, blob=object, bucket=bucket) - else: - return self.download_as_string(blob=object, bucket=bucket) - - def upload( - self, - bucket, - object, - filename, - mime_type='application/octet-stream'): - """ - This method is provided for compatibility with - contrib/GoogleCloudStorageHook. - - Warning: acts as if replace == True! - """ - self.upload_from_filename( - filename=filename, blob=object, bucket=bucket, replace=True) diff --git a/airflow/contrib/hooks/gcloud/readme.md b/airflow/contrib/hooks/gcloud/readme.md deleted file mode 100644 index 6a6905cb9c708..0000000000000 --- a/airflow/contrib/hooks/gcloud/readme.md +++ /dev/null @@ -1,20 +0,0 @@ -## GCP interfaces for Airflow - -Please note: Airflow currently has two sets of hooks/operators for interacting with the Google Cloud Platform. At this time (March 2016), they are not compatible with each other due to reliance on different versions of the `oauth2client` library (which introduced breaking changes with version 2.0 in February 2016). Therefore, users have to install them explicitly. - -##### The `gcp_api` package -The first set is based on a version of the [Google API Client](https://github.com/google/google-api-python-client) (`google-api-python-client`) that requires `oauth2client < 2.0`. It includes a large number of hooks/operators covering GCS, DataStore, and BigQuery. - -To use this set, install `airflow[gcp_api]`. - -##### The `gcloud` package -The second set is based on the [`gcloud` Python library](https://googlecloudplatform.github.io/gcloud-python/) and requires `oauth2client >= 2.0`. At this time it only includes a hook for GCS. Note that the hooks/operators in this set live in `gcloud/` directories, for clarity. - -To use this set, install `airflow[gcloud]`. - -##### Which should I use? -New users should probably build on the `gcloud` set because the `gcloud` library is the recommended way for Python apps to interact with the Google Cloud Platform. The interface is easier to extend than the API approach. - -More pragmatically, if your existing code (hooks/operators/DAGs) depends on EITHER `gcloud >= 0.10` OR `google-api-python-client >= 1.5` (which both require `oauth2client >= 2.0`), then you won't be able to use `airflow[gcp_api]` due to compatibility issues. - -However, if the hooks/operators in the `gcp_api` set meet your needs and you do not have other dependencies, then by all means use them! diff --git a/airflow/contrib/hooks/gcs_hook.py b/airflow/contrib/hooks/gcs_hook.py index c59facf454043..39d7be7545617 100644 --- a/airflow/contrib/hooks/gcs_hook.py +++ b/airflow/contrib/hooks/gcs_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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 httplib2 import logging diff --git a/airflow/contrib/hooks/qubole_hook.py b/airflow/contrib/hooks/qubole_hook.py index c36b9f5838f5e..7308b64a8a784 100755 --- a/airflow/contrib/hooks/qubole_hook.py +++ b/airflow/contrib/hooks/qubole_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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 os import time import datetime diff --git a/airflow/contrib/hooks/vertica_hook.py b/airflow/contrib/hooks/vertica_hook.py index 1acf653c94fb1..9d9ae1db08e27 100644 --- a/airflow/contrib/hooks/vertica_hook.py +++ b/airflow/contrib/hooks/vertica_hook.py @@ -1,3 +1,18 @@ +# -*- coding: utf-8 -*- +# +# 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. +# + from vertica_python import connect from airflow.hooks.dbapi_hook import DbApiHook diff --git a/airflow/utils/logging.py b/airflow/utils/logging.py index edf33c8801d1a..5fe876a99db4a 100644 --- a/airflow/utils/logging.py +++ b/airflow/utils/logging.py @@ -116,36 +116,27 @@ def write(self, log, remote_log_location, append=False): class GCSLog(object): """ - Utility class for reading and writing logs in GCS. - Requires either airflow[gcloud] or airflow[gcp_api] and - setting the REMOTE_BASE_LOG_FOLDER and REMOTE_LOG_CONN_ID configuration - options in airflow.cfg. + Utility class for reading and writing logs in GCS. Requires + airflow[gcp_api] and setting the REMOTE_BASE_LOG_FOLDER and + REMOTE_LOG_CONN_ID configuration options in airflow.cfg. """ def __init__(self): """ - Attempt to create hook with airflow[gcloud] (and set - use_gcloud = True), otherwise uses airflow[gcp_api] + Attempt to create hook with airflow[gcp_api]. """ remote_conn_id = configuration.get('core', 'REMOTE_LOG_CONN_ID') - self.use_gcloud = False + self.hook = None try: - from airflow.contrib.hooks import GCSHook - self.hook = GCSHook(remote_conn_id) - self.use_gcloud = True + from airflow.contrib.hooks import GoogleCloudStorageHook + self.hook = GoogleCloudStorageHook( + scope='https://www.googleapis.com/auth/devstorage.read_write', + google_cloud_storage_conn_id=remote_conn_id) except: - try: - from airflow.contrib.hooks import GoogleCloudStorageHook - self.hook = GoogleCloudStorageHook( - scope='https://www.googleapis.com/auth/devstorage.read_write', - google_cloud_storage_conn_id=remote_conn_id) - except: - self.hook = None - logging.error( - 'Could not create a GCSHook with connection id "{}". ' - 'Please make sure that either airflow[gcloud] or ' - 'airflow[gcp_api] is installed and the GCS connection ' - 'exists.'.format(remote_conn_id)) + logging.error( + 'Could not create a GoogleCloudStorageHook with connection id ' + '"{}". Please make sure that airflow[gcp_api] is installed ' + 'and the GCS connection exists.'.format(remote_conn_id)) def read(self, remote_log_location, return_error=False): """ @@ -159,13 +150,8 @@ def read(self, remote_log_location, return_error=False): """ if self.hook: try: - if self.use_gcloud: - gcs_blob = self.hook.get_blob(remote_log_location) - if gcs_blob: - return gcs_blob.download_as_string().decode() - else: - bkt, blob = self.parse_gcs_url(remote_log_location) - return self.hook.download(bkt, blob).decode() + bkt, blob = self.parse_gcs_url(remote_log_location) + return self.hook.download(bkt, blob).decode() except: pass @@ -189,34 +175,23 @@ def write(self, log, remote_log_location, append=False): """ if self.hook: - if append: old_log = self.read(remote_log_location) log = old_log + '\n' + log try: - if self.use_gcloud: - self.hook.upload_from_string( - log, - blob=remote_log_location, - replace=True) - return - else: - bkt, blob = self.parse_gcs_url(remote_log_location) - from tempfile import NamedTemporaryFile - with NamedTemporaryFile(mode='w+') as tmpfile: - tmpfile.write(log) - # Force the file to be flushed, since we're doing the - # upload from within the file context (it hasn't been - # closed). - tmpfile.flush() - self.hook.upload(bkt, blob, tmpfile.name) - return + bkt, blob = self.parse_gcs_url(remote_log_location) + from tempfile import NamedTemporaryFile + with NamedTemporaryFile(mode='w+') as tmpfile: + tmpfile.write(log) + # Force the file to be flushed, since we're doing the + # upload from within the file context (it hasn't been + # closed). + tmpfile.flush() + self.hook.upload(bkt, blob, tmpfile.name) except: - pass - - # raise/return error if we get here - logging.error('Could not write logs to {}'.format(remote_log_location)) + # raise/return error if we get here + logging.error('Could not write logs to {}'.format(remote_log_location)) def parse_gcs_url(self, gsurl): """ diff --git a/docs/installation.rst b/docs/installation.rst index 0023ee303b38e..289f64ff6b780 100644 --- a/docs/installation.rst +++ b/docs/installation.rst @@ -52,10 +52,6 @@ Here's the list of the subpackages and what they enable: | gcp_api | ``pip install airflow[gcp_api]`` | Google Cloud Platform hooks and operators | | | | (using ``google-api-python-client``) | +---------------+-------------------------------------+-------------------------------------------------+ -| gcloud | ``pip install airflow[gcloud]`` | Google Cloud Platform hooks | -| | | (using ``gcloud``; | -| | | see ``/airflow/contrib/hooks/gcloud/readme.md``)| -+---------------+-------------------------------------+-------------------------------------------------+ | jdbc | ``pip install airflow[jdbc]`` | JDBC hooks and operators | +---------------+-------------------------------------+-------------------------------------------------+ | hdfs | ``pip install airflow[hdfs]`` | HDFS hooks and operators | diff --git a/setup.py b/setup.py index d8a1adb3e54b6..c4c5eeeed9b44 100644 --- a/setup.py +++ b/setup.py @@ -53,9 +53,6 @@ def run(self): ] docker = ['docker-py>=1.6.0'] druid = ['pydruid>=0.2.1'] -gcloud = [ - 'gcloud>=0.11.0', -] gcp_api = [ 'httplib2', 'google-api-python-client<=1.4.2', @@ -149,7 +146,6 @@ def run(self): 'doc': doc, 'docker': docker, 'druid': druid, - 'gcloud': gcloud, 'gcp_api': gcp_api, 'hdfs': hdfs, 'hive': hive,