You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2019/01/11 16:27:23 UTC

[GitHub] potiuk closed pull request #4486: [AIRFLOW-3681] Optional project id for all operators

potiuk closed pull request #4486: [AIRFLOW-3681] Optional project id for all operators
URL: https://github.com/apache/airflow/pull/4486
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/contrib/example_dags/example_gcp_bigtable_operators.py b/airflow/contrib/example_dags/example_gcp_bigtable_operators.py
index 48c4245cba..7ab9e02e0a 100644
--- a/airflow/contrib/example_dags/example_gcp_bigtable_operators.py
+++ b/airflow/contrib/example_dags/example_gcp_bigtable_operators.py
@@ -17,6 +17,7 @@
 # specific language governing permissions and limitations
 # under the License.
 
+# noinspection LongLine
 """
 Example Airflow DAG that creates and performs following operations on Cloud Bigtable:
 - creates an Instance
@@ -26,12 +27,13 @@
 - deletes the Table
 - deletes the Instance
 
-This DAG relies on the following environment variables
+This DAG relies on the following environment variables:
+
 * GCP_PROJECT_ID - Google Cloud Platform project
 * CBT_INSTANCE_ID - desired ID of a Cloud Bigtable instance
 * CBT_INSTANCE_DISPLAY_NAME - desired human-readable display name of the Instance
 * CBT_INSTANCE_TYPE - type of the Instance, e.g. 1 for DEVELOPMENT
-    See https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance
+    See https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance # noqa E501
 * CBT_INSTANCE_LABELS - labels to add for the Instance
 * CBT_CLUSTER_ID - desired ID of the main Cluster created for the Instance
 * CBT_CLUSTER_ZONE - zone in which main Cluster will be created. e.g. europe-west1-b
@@ -39,22 +41,25 @@
 * CBT_CLUSTER_NODES - initial amount of nodes of the Cluster
 * CBT_CLUSTER_NODES_UPDATED - amount of nodes for BigtableClusterUpdateOperator
 * CBT_CLUSTER_STORAGE_TYPE - storage for the Cluster, e.g. 1 for SSD
-    See https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.cluster # noqa: E501
+    See https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.cluster # noqa E501
 * CBT_TABLE_ID - desired ID of the Table
 * CBT_POKE_INTERVAL - number of seconds between every attempt of Sensor check
 
 """
 
-import datetime
 import json
 
 from os import getenv
 
 import airflow
 from airflow import models
-from airflow.contrib.operators.gcp_bigtable_operator import BigtableInstanceCreateOperator, \
-    BigtableInstanceDeleteOperator, BigtableClusterUpdateOperator, BigtableTableCreateOperator, \
-    BigtableTableWaitForReplicationSensor, BigtableTableDeleteOperator
+from airflow.contrib.operators.gcp_bigtable_operator import \
+    BigtableInstanceCreateOperator, \
+    BigtableInstanceDeleteOperator, \
+    BigtableClusterUpdateOperator, \
+    BigtableTableCreateOperator, \
+    BigtableTableWaitForReplicationSensor, \
+    BigtableTableDeleteOperator
 
 # [START howto_operator_gcp_bigtable_args]
 GCP_PROJECT_ID = getenv('GCP_PROJECT_ID', 'example-project')
@@ -78,7 +83,7 @@
 with models.DAG(
     'example_gcp_bigtable_operators',
     default_args=default_args,
-    schedule_interval=datetime.timedelta(days=1)
+    schedule_interval=None  # Override to match your needs
 ) as dag:
     # [START howto_operator_gcp_bigtable_instance_create]
     create_instance_task = BigtableInstanceCreateOperator(
@@ -91,8 +96,20 @@
         instance_labels=json.loads(CBT_INSTANCE_LABELS),
         cluster_nodes=int(CBT_CLUSTER_NODES),
         cluster_storage_type=int(CBT_CLUSTER_STORAGE_TYPE),
-        task_id='create_instance',
+        task_id='create_instance_task',
+    )
+    create_instance_task2 = BigtableInstanceCreateOperator(
+        instance_id=CBT_INSTANCE_ID,
+        main_cluster_id=CBT_CLUSTER_ID,
+        main_cluster_zone=CBT_CLUSTER_ZONE,
+        instance_display_name=CBT_INSTANCE_DISPLAY_NAME,
+        instance_type=int(CBT_INSTANCE_TYPE),
+        instance_labels=json.loads(CBT_INSTANCE_LABELS),
+        cluster_nodes=int(CBT_CLUSTER_NODES),
+        cluster_storage_type=int(CBT_CLUSTER_STORAGE_TYPE),
+        task_id='create_instance_task2',
     )
+    create_instance_task >> create_instance_task2
     # [END howto_operator_gcp_bigtable_instance_create]
 
     # [START howto_operator_gcp_bigtable_cluster_update]
@@ -101,15 +118,26 @@
         instance_id=CBT_INSTANCE_ID,
         cluster_id=CBT_CLUSTER_ID,
         nodes=int(CBT_CLUSTER_NODES_UPDATED),
-        task_id='update_cluster',
+        task_id='update_cluster_task',
+    )
+    cluster_update_task2 = BigtableClusterUpdateOperator(
+        instance_id=CBT_INSTANCE_ID,
+        cluster_id=CBT_CLUSTER_ID,
+        nodes=int(CBT_CLUSTER_NODES_UPDATED),
+        task_id='update_cluster_task2',
     )
+    cluster_update_task >> cluster_update_task2
     # [END howto_operator_gcp_bigtable_cluster_update]
 
     # [START howto_operator_gcp_bigtable_instance_delete]
     delete_instance_task = BigtableInstanceDeleteOperator(
         project_id=GCP_PROJECT_ID,
         instance_id=CBT_INSTANCE_ID,
-        task_id='delete_instance',
+        task_id='delete_instance_task',
+    )
+    delete_instance_task2 = BigtableInstanceDeleteOperator(
+        instance_id=CBT_INSTANCE_ID,
+        task_id='delete_instance_task2',
     )
     # [END howto_operator_gcp_bigtable_instance_delete]
 
@@ -120,6 +148,12 @@
         table_id=CBT_TABLE_ID,
         task_id='create_table',
     )
+    create_table_task2 = BigtableTableCreateOperator(
+        instance_id=CBT_INSTANCE_ID,
+        table_id=CBT_TABLE_ID,
+        task_id='create_table_task2',
+    )
+    create_table_task >> create_table_task2
     # [END howto_operator_gcp_bigtable_table_create]
 
     # [START howto_operator_gcp_bigtable_table_wait_for_replication]
@@ -128,7 +162,15 @@
         instance_id=CBT_INSTANCE_ID,
         table_id=CBT_TABLE_ID,
         poke_interval=int(CBT_POKE_INTERVAL),
-        task_id='wait_for_table_replication',
+        timeout=180,
+        task_id='wait_for_table_replication_task',
+    )
+    wait_for_table_replication_task2 = BigtableTableWaitForReplicationSensor(
+        instance_id=CBT_INSTANCE_ID,
+        table_id=CBT_TABLE_ID,
+        poke_interval=int(CBT_POKE_INTERVAL),
+        timeout=180,
+        task_id='wait_for_table_replication_task2',
     )
     # [END howto_operator_gcp_bigtable_table_wait_for_replication]
 
@@ -137,13 +179,28 @@
         project_id=GCP_PROJECT_ID,
         instance_id=CBT_INSTANCE_ID,
         table_id=CBT_TABLE_ID,
-        task_id='delete_table',
+        task_id='delete_table_task',
+    )
+    delete_table_task2 = BigtableTableDeleteOperator(
+        instance_id=CBT_INSTANCE_ID,
+        table_id=CBT_TABLE_ID,
+        task_id='delete_table_task2',
     )
     # [END howto_operator_gcp_bigtable_table_delete]
 
     wait_for_table_replication_task >> delete_table_task
+    wait_for_table_replication_task2 >> delete_table_task
+    wait_for_table_replication_task >> delete_table_task2
+    wait_for_table_replication_task2 >> delete_table_task2
     create_instance_task \
         >> create_table_task \
         >> cluster_update_task \
-        >> delete_table_task \
-        >> delete_instance_task
+        >> delete_table_task
+    create_instance_task2 \
+        >> create_table_task2 \
+        >> cluster_update_task2 \
+        >> delete_table_task2
+
+    # Only delete instances after all tables are deleted
+    [delete_table_task, delete_table_task2] >> \
+        delete_instance_task >> delete_instance_task2
diff --git a/airflow/contrib/example_dags/example_gcp_compute.py b/airflow/contrib/example_dags/example_gcp_compute.py
index 928e9744b6..dde0f5975a 100644
--- a/airflow/contrib/example_dags/example_gcp_compute.py
+++ b/airflow/contrib/example_dags/example_gcp_compute.py
@@ -23,14 +23,13 @@
 
 This DAG relies on the following OS environment variables
 
-* PROJECT_ID - Google Cloud Platform project where the Compute Engine instance exists.
-* ZONE - Google Cloud Platform zone where the instance exists.
-* INSTANCE - Name of the Compute Engine instance.
-* SHORT_MACHINE_TYPE_NAME - Machine type resource name to set, e.g. 'n1-standard-1'.
+* GCP_PROJECT_ID - Google Cloud Platform project where the Compute Engine instance exists.
+* GCE_ZONE - Google Cloud Platform zone where the instance exists.
+* GCE_INSTANCE - Name of the Compute Engine instance.
+* GCE_SHORT_MACHINE_TYPE_NAME - Machine type resource name to set, e.g. 'n1-standard-1'.
     See https://cloud.google.com/compute/docs/machine-types
 """
 import os
-import datetime
 
 import airflow
 from airflow import models
@@ -38,19 +37,19 @@
     GceInstanceStopOperator, GceSetMachineTypeOperator
 
 # [START howto_operator_gce_args_common]
-PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project')
-ZONE = os.environ.get('ZONE', 'europe-west1-b')
-INSTANCE = os.environ.get('INSTANCE', 'testinstance')
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCE_ZONE = os.environ.get('GCE_ZONE', 'europe-west1-b')
+GCE_INSTANCE = os.environ.get('GCE_INSTANCE', 'testinstance')
 # [END howto_operator_gce_args_common]
 
 default_args = {
-    'start_date': airflow.utils.dates.days_ago(1)
+    'start_date': airflow.utils.dates.days_ago(1),
 }
 
 # [START howto_operator_gce_args_set_machine_type]
-SHORT_MACHINE_TYPE_NAME = os.environ.get('SHORT_MACHINE_TYPE_NAME', 'n1-standard-1')
+GCE_SHORT_MACHINE_TYPE_NAME = os.environ.get('GCE_SHORT_MACHINE_TYPE_NAME', 'n1-standard-1')
 SET_MACHINE_TYPE_BODY = {
-    'machineType': 'zones/{}/machineTypes/{}'.format(ZONE, SHORT_MACHINE_TYPE_NAME)
+    'machineType': 'zones/{}/machineTypes/{}'.format(GCE_ZONE, GCE_SHORT_MACHINE_TYPE_NAME)
 }
 # [END howto_operator_gce_args_set_machine_type]
 
@@ -58,55 +57,58 @@
 with models.DAG(
     'example_gcp_compute',
     default_args=default_args,
-    schedule_interval=datetime.timedelta(days=1)
+    schedule_interval=None  # Override to match your needs
 ) as dag:
     # [START howto_operator_gce_start]
     gce_instance_start = GceInstanceStartOperator(
-        project_id=PROJECT_ID,
-        zone=ZONE,
-        resource_id=INSTANCE,
+        project_id=GCP_PROJECT_ID,
+        zone=GCE_ZONE,
+        resource_id=GCE_INSTANCE,
         task_id='gcp_compute_start_task'
     )
     # [END howto_operator_gce_start]
     # Duplicate start for idempotence testing
+    # [START howto_operator_gce_start_no_project_id]
     gce_instance_start2 = GceInstanceStartOperator(
-        project_id=PROJECT_ID,
-        zone=ZONE,
-        resource_id=INSTANCE,
+        zone=GCE_ZONE,
+        resource_id=GCE_INSTANCE,
         task_id='gcp_compute_start_task2'
     )
+    # [END howto_operator_gce_start_no_project_id]
     # [START howto_operator_gce_stop]
     gce_instance_stop = GceInstanceStopOperator(
-        project_id=PROJECT_ID,
-        zone=ZONE,
-        resource_id=INSTANCE,
+        project_id=GCP_PROJECT_ID,
+        zone=GCE_ZONE,
+        resource_id=GCE_INSTANCE,
         task_id='gcp_compute_stop_task'
     )
     # [END howto_operator_gce_stop]
     # Duplicate stop for idempotence testing
+    # [START howto_operator_gce_stop_no_project_id]
     gce_instance_stop2 = GceInstanceStopOperator(
-        project_id=PROJECT_ID,
-        zone=ZONE,
-        resource_id=INSTANCE,
+        zone=GCE_ZONE,
+        resource_id=GCE_INSTANCE,
         task_id='gcp_compute_stop_task2'
     )
+    # [END howto_operator_gce_stop_no_project_id]
     # [START howto_operator_gce_set_machine_type]
     gce_set_machine_type = GceSetMachineTypeOperator(
-        project_id=PROJECT_ID,
-        zone=ZONE,
-        resource_id=INSTANCE,
+        project_id=GCP_PROJECT_ID,
+        zone=GCE_ZONE,
+        resource_id=GCE_INSTANCE,
         body=SET_MACHINE_TYPE_BODY,
         task_id='gcp_compute_set_machine_type'
     )
     # [END howto_operator_gce_set_machine_type]
     # Duplicate set machine type for idempotence testing
+    # [START howto_operator_gce_set_machine_type_no_project_id]
     gce_set_machine_type2 = GceSetMachineTypeOperator(
-        project_id=PROJECT_ID,
-        zone=ZONE,
-        resource_id=INSTANCE,
+        zone=GCE_ZONE,
+        resource_id=GCE_INSTANCE,
         body=SET_MACHINE_TYPE_BODY,
         task_id='gcp_compute_set_machine_type2'
     )
+    # [END howto_operator_gce_set_machine_type_no_project_id]
 
     gce_instance_start >> gce_instance_start2 >> gce_instance_stop >> \
         gce_instance_stop2 >> gce_set_machine_type >> gce_set_machine_type2
diff --git a/airflow/contrib/example_dags/example_gcp_compute_igm.py b/airflow/contrib/example_dags/example_gcp_compute_igm.py
index 3e4543c60d..103cbfd590 100644
--- a/airflow/contrib/example_dags/example_gcp_compute_igm.py
+++ b/airflow/contrib/example_dags/example_gcp_compute_igm.py
@@ -24,23 +24,22 @@
 
 This DAG relies on the following OS environment variables
 
-* PROJECT_ID - the Google Cloud Platform project where the Compute Engine instance exists
-* ZONE - the zone where the Compute Engine instance exists
+* GCP_PROJECT_ID - the Google Cloud Platform project where the Compute Engine instance exists
+* GCE_ZONE - the zone where the Compute Engine instance exists
 
 Variables for copy template operator:
-* TEMPLATE_NAME - name of the template to copy
-* NEW_TEMPLATE_NAME - name of the new template
-* NEW_DESCRIPTION - description added to the template
+* GCE_TEMPLATE_NAME - name of the template to copy
+* GCE_NEW_TEMPLATE_NAME - name of the new template
+* GCE_NEW_DESCRIPTION - description added to the template
 
 Variables for update template in Group Manager:
 
-* INSTANCE_GROUP_MANAGER_NAME - name of the Instance Group Manager
+* GCE_INSTANCE_GROUP_MANAGER_NAME - name of the Instance Group Manager
 * SOURCE_TEMPLATE_URL - url of the template to replace in the Instance Group Manager
 * DESTINATION_TEMPLATE_URL - url of the new template to set in the Instance Group Manager
 """
 
 import os
-import datetime
 
 import airflow
 from airflow import models
@@ -48,8 +47,8 @@
     GceInstanceTemplateCopyOperator, GceInstanceGroupManagerUpdateTemplateOperator
 
 # [START howto_operator_compute_igm_common_args]
-PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project')
-ZONE = os.environ.get('ZONE', 'europe-west1-b')
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCE_ZONE = os.environ.get('GCE_ZONE', 'europe-west1-b')
 # [END howto_operator_compute_igm_common_args]
 
 default_args = {
@@ -57,13 +56,13 @@
 }
 
 # [START howto_operator_compute_template_copy_args]
-TEMPLATE_NAME = os.environ.get('TEMPLATE_NAME', 'instance-template-test')
-NEW_TEMPLATE_NAME = os.environ.get('NEW_TEMPLATE_NAME',
-                                   'instance-template-test-new')
-NEW_DESCRIPTION = os.environ.get('NEW_DESCRIPTION', 'Test new description')
+GCE_TEMPLATE_NAME = os.environ.get('GCE_TEMPLATE_NAME', 'instance-template-test')
+GCE_NEW_TEMPLATE_NAME = os.environ.get('GCE_NEW_TEMPLATE_NAME',
+                                       'instance-template-test-new')
+GCE_NEW_DESCRIPTION = os.environ.get('GCE_NEW_DESCRIPTION', 'Test new description')
 GCE_INSTANCE_TEMPLATE_BODY_UPDATE = {
-    "name": NEW_TEMPLATE_NAME,
-    "description": NEW_DESCRIPTION,
+    "name": GCE_NEW_TEMPLATE_NAME,
+    "description": GCE_NEW_DESCRIPTION,
     "properties": {
         "machineType": "n1-standard-2"
     }
@@ -71,18 +70,18 @@
 # [END howto_operator_compute_template_copy_args]
 
 # [START howto_operator_compute_igm_update_template_args]
-INSTANCE_GROUP_MANAGER_NAME = os.environ.get('INSTANCE_GROUP_MANAGER_NAME',
-                                             'instance-group-test')
+GCE_INSTANCE_GROUP_MANAGER_NAME = os.environ.get('GCE_INSTANCE_GROUP_MANAGER_NAME',
+                                                 'instance-group-test')
 
 SOURCE_TEMPLATE_URL = os.environ.get(
     'SOURCE_TEMPLATE_URL',
-    "https://www.googleapis.com/compute/beta/projects/"
-    "example-project/global/instanceTemplates/instance-template-test")
+    "https://www.googleapis.com/compute/beta/projects/" + GCP_PROJECT_ID +
+    "/global/instanceTemplates/instance-template-test")
 
 DESTINATION_TEMPLATE_URL = os.environ.get(
     'DESTINATION_TEMPLATE_URL',
-    "https://www.googleapis.com/compute/beta/projects/"
-    "example-airflow/global/instanceTemplates/" + NEW_TEMPLATE_NAME)
+    "https://www.googleapis.com/compute/beta/projects/" + GCP_PROJECT_ID +
+    "/global/instanceTemplates/" + GCE_NEW_TEMPLATE_NAME)
 
 UPDATE_POLICY = {
     "type": "OPPORTUNISTIC",
@@ -99,29 +98,30 @@
 with models.DAG(
     'example_gcp_compute_igm',
     default_args=default_args,
-    schedule_interval=datetime.timedelta(days=1)
+    schedule_interval=None  # Override to match your needs
 ) as dag:
     # [START howto_operator_gce_igm_copy_template]
     gce_instance_template_copy = GceInstanceTemplateCopyOperator(
-        project_id=PROJECT_ID,
-        resource_id=TEMPLATE_NAME,
+        project_id=GCP_PROJECT_ID,
+        resource_id=GCE_TEMPLATE_NAME,
         body_patch=GCE_INSTANCE_TEMPLATE_BODY_UPDATE,
         task_id='gcp_compute_igm_copy_template_task'
     )
     # [END howto_operator_gce_igm_copy_template]
     # Added to check for idempotence
+    # [START howto_operator_gce_igm_copy_template_no_project_id]
     gce_instance_template_copy2 = GceInstanceTemplateCopyOperator(
-        project_id=PROJECT_ID,
-        resource_id=TEMPLATE_NAME,
+        resource_id=GCE_TEMPLATE_NAME,
         body_patch=GCE_INSTANCE_TEMPLATE_BODY_UPDATE,
         task_id='gcp_compute_igm_copy_template_task_2'
     )
+    # [END howto_operator_gce_igm_copy_template_no_project_id]
     # [START howto_operator_gce_igm_update_template]
     gce_instance_group_manager_update_template = \
         GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            resource_id=INSTANCE_GROUP_MANAGER_NAME,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
+            zone=GCE_ZONE,
             source_template=SOURCE_TEMPLATE_URL,
             destination_template=DESTINATION_TEMPLATE_URL,
             update_policy=UPDATE_POLICY,
@@ -129,15 +129,16 @@
         )
     # [END howto_operator_gce_igm_update_template]
     # Added to check for idempotence (and without UPDATE_POLICY)
+    # [START howto_operator_gce_igm_update_template_no_project_id]
     gce_instance_group_manager_update_template2 = \
         GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            resource_id=INSTANCE_GROUP_MANAGER_NAME,
-            zone=ZONE,
+            resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
+            zone=GCE_ZONE,
             source_template=SOURCE_TEMPLATE_URL,
             destination_template=DESTINATION_TEMPLATE_URL,
             task_id='gcp_compute_igm_group_manager_update_template_2'
         )
+    # [END howto_operator_gce_igm_update_template_no_project_id]
     gce_instance_template_copy >> gce_instance_template_copy2 >> \
         gce_instance_group_manager_update_template >> \
         gce_instance_group_manager_update_template2
diff --git a/airflow/contrib/example_dags/example_gcp_function.py b/airflow/contrib/example_dags/example_gcp_function.py
new file mode 100644
index 0000000000..b24f345544
--- /dev/null
+++ b/airflow/contrib/example_dags/example_gcp_function.py
@@ -0,0 +1,133 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+
+"""
+Example Airflow DAG that displays interactions with Google Cloud Functions.
+It creates a function, invokes and then deletes it.
+
+This DAG relies on the following OS environment variables
+https://airflow.apache.org/concepts.html#variables
+* GCP_PROJECT_ID - Google Cloud Project to use for the Cloud Function.
+* GCP_LOCATION - Google Cloud Functions region where the function should be
+  created.
+* GCF_ENTRYPOINT - Name of the executable function in the source code.
+* and one of the below:
+    - GCF_SOURCE_ARCHIVE_URL - Path to the zipped source in Google Cloud Storage
+    or
+    (
+        - GCF_SOURCE_UPLOAD_URL - Generated upload URL for the zipped source
+        and
+        - GCF_ZIP_PATH - Local path to the zipped source archive
+    )
+    or
+    - GCF_SOURCE_REPOSITORY - The URL pointing to the hosted repository where the function
+    is defined in a supported Cloud Source Repository URL format
+    https://cloud.google.com/functions/docs/reference/rest/v1/projects.locations.functions#SourceRepository
+"""
+
+import os
+
+from airflow import models
+from airflow.contrib.operators.gcp_function_operator \
+    import GcfFunctionDeployOperator, GcfFunctionDeleteOperator
+from airflow.utils import dates
+
+# [START howto_operator_gcf_common_variables]
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCP_LOCATION = os.environ.get('GCP_LOCATION', 'europe-west1')
+GCF_SHORT_FUNCTION_NAME = os.environ.get('GCF_SHORT_FUNCTION_NAME', 'hello').\
+    replace("-", "_")  # make sure there are no dashes in function name (!)
+FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(GCP_PROJECT_ID,
+                                                               GCP_LOCATION,
+                                                               GCF_SHORT_FUNCTION_NAME)
+# [END howto_operator_gcf_common_variables]
+# [START howto_operator_gcf_deploy_variables]
+GCF_SOURCE_ARCHIVE_URL = os.environ.get('GCF_SOURCE_ARCHIVE_URL', '')
+GCF_SOURCE_UPLOAD_URL = os.environ.get('GCF_SOURCE_UPLOAD_URL', '')
+GCF_SOURCE_REPOSITORY = os.environ.get(
+    'GCF_SOURCE_REPOSITORY',
+    'https://source.developers.google.com/'
+    'projects/{}/repos/hello-world/moveable-aliases/master'.format(GCP_PROJECT_ID))
+GCF_ZIP_PATH = os.environ.get('GCF_ZIP_PATH', '')
+GCF_ENTRYPOINT = os.environ.get('GCF_ENTRYPOINT', 'helloWorld')
+GCF_RUNTIME = 'nodejs6'
+GCP_VALIDATE_BODY = os.environ.get('GCP_VALIDATE_BODY', True)
+# [END howto_operator_gcf_deploy_variables]
+
+# [START howto_operator_gcf_deploy_body]
+body = {
+    "name": FUNCTION_NAME,
+    "entryPoint": GCF_ENTRYPOINT,
+    "runtime": GCF_RUNTIME,
+    "httpsTrigger": {}
+}
+# [END howto_operator_gcf_deploy_body]
+
+# [START howto_operator_gcf_default_args]
+default_args = {
+    'start_date': dates.days_ago(1)
+}
+# [END howto_operator_gcf_default_args]
+
+# [START howto_operator_gcf_deploy_variants]
+if GCF_SOURCE_ARCHIVE_URL:
+    body['sourceArchiveUrl'] = GCF_SOURCE_ARCHIVE_URL
+elif GCF_SOURCE_REPOSITORY:
+    body['sourceRepository'] = {
+        'url': GCF_SOURCE_REPOSITORY
+    }
+elif GCF_ZIP_PATH:
+    body['sourceUploadUrl'] = ''
+    default_args['zip_path'] = GCF_ZIP_PATH
+elif GCF_SOURCE_UPLOAD_URL:
+    body['sourceUploadUrl'] = GCF_SOURCE_UPLOAD_URL
+else:
+    raise Exception("Please provide one of the source_code parameters")
+# [END howto_operator_gcf_deploy_variants]
+
+
+with models.DAG(
+    'example_gcp_function',
+    default_args=default_args,
+    schedule_interval=None  # Override to match your needs
+) as dag:
+    # [START howto_operator_gcf_deploy]
+    deploy_task = GcfFunctionDeployOperator(
+        task_id="gcf_deploy_task",
+        project_id=GCP_PROJECT_ID,
+        location=GCP_LOCATION,
+        body=body,
+        validate_body=GCP_VALIDATE_BODY
+    )
+    # [END howto_operator_gcf_deploy]
+    # [START howto_operator_gcf_deploy_no_project_id]
+    deploy2_task = GcfFunctionDeployOperator(
+        task_id="gcf_deploy2_task",
+        location=GCP_LOCATION,
+        body=body,
+        validate_body=GCP_VALIDATE_BODY
+    )
+    # [END howto_operator_gcf_deploy_no_project_id]
+    # [START howto_operator_gcf_delete]
+    delete_task = GcfFunctionDeleteOperator(
+        task_id="gcf_delete_task",
+        name=FUNCTION_NAME
+    )
+    # [END howto_operator_gcf_delete]
+    deploy_task >> deploy2_task >> delete_task
diff --git a/airflow/contrib/example_dags/example_gcp_function_delete.py b/airflow/contrib/example_dags/example_gcp_function_delete.py
deleted file mode 100644
index 642e3a744c..0000000000
--- a/airflow/contrib/example_dags/example_gcp_function_delete.py
+++ /dev/null
@@ -1,59 +0,0 @@
-# -*- coding: utf-8 -*-
-#
-# 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.
-
-"""
-Example Airflow DAG that deletes a Google Cloud Function.
-This DAG relies on the following OS environment variables
-* PROJECT_ID - Google Cloud Project where the Cloud Function exists.
-* LOCATION - Google Cloud Functions region where the function exists.
-* ENTRYPOINT - Name of the executable function in the source code.
-"""
-
-import os
-import datetime
-
-import airflow
-from airflow import models
-from airflow.contrib.operators.gcp_function_operator import GcfFunctionDeleteOperator
-
-# [START howto_operator_gcf_delete_args]
-PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project')
-LOCATION = os.environ.get('LOCATION', 'europe-west1')
-ENTRYPOINT = os.environ.get('ENTRYPOINT', 'helloWorld')
-# A fully-qualified name of the function to delete
-
-FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(PROJECT_ID, LOCATION,
-                                                               ENTRYPOINT)
-# [END howto_operator_gcf_delete_args]
-
-default_args = {
-    'start_date': airflow.utils.dates.days_ago(1)
-}
-
-with models.DAG(
-    'example_gcp_function_delete',
-    default_args=default_args,
-    schedule_interval=datetime.timedelta(days=1)
-) as dag:
-    # [START howto_operator_gcf_delete]
-    t1 = GcfFunctionDeleteOperator(
-        task_id="gcf_delete_task",
-        name=FUNCTION_NAME
-    )
-    # [END howto_operator_gcf_delete]
diff --git a/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py b/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
deleted file mode 100644
index 76563d7596..0000000000
--- a/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
+++ /dev/null
@@ -1,117 +0,0 @@
-# -*- coding: utf-8 -*-
-#
-# 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.
-
-"""
-Example Airflow DAG that creates a Google Cloud Function and then deletes it.
-
-This DAG relies on the following OS environment variables
-https://airflow.apache.org/concepts.html#variables
-* PROJECT_ID - Google Cloud Project to use for the Cloud Function.
-* LOCATION - Google Cloud Functions region where the function should be
-  created.
-* SOURCE_ARCHIVE_URL - Path to the zipped source in Google Cloud Storage
-or
-    * SOURCE_UPLOAD_URL - Generated upload URL for the zipped source
-    or
-    * ZIP_PATH - Local path to the zipped source archive
-or
-* SOURCE_REPOSITORY - The URL pointing to the hosted repository where the function is
-defined in a supported Cloud Source Repository URL format
-https://cloud.google.com/functions/docs/reference/rest/v1/projects.locations.functions#SourceRepository
-* ENTRYPOINT - Name of the executable function in the source code.
-"""
-
-import os
-import datetime
-
-from airflow import models
-from airflow.contrib.operators.gcp_function_operator \
-    import GcfFunctionDeployOperator, GcfFunctionDeleteOperator
-from airflow.utils import dates
-
-# [START howto_operator_gcf_deploy_variables]
-PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project')
-LOCATION = os.environ.get('LOCATION', 'europe-west1')
-SOURCE_ARCHIVE_URL = os.environ.get('SOURCE_ARCHIVE_URL', '')
-SOURCE_UPLOAD_URL = os.environ.get('SOURCE_UPLOAD_URL', '')
-SOURCE_REPOSITORY = os.environ.get(
-    'SOURCE_REPOSITORY',
-    'https://source.developers.google.com/'
-    'projects/example-project/repos/hello-world/moveable-aliases/master')
-ZIP_PATH = os.environ.get('ZIP_PATH', '')
-ENTRYPOINT = os.environ.get('ENTRYPOINT', 'helloWorld')
-FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(PROJECT_ID, LOCATION,
-                                                               ENTRYPOINT)
-RUNTIME = 'nodejs6'
-VALIDATE_BODY = os.environ.get('VALIDATE_BODY', True)
-
-# [END howto_operator_gcf_deploy_variables]
-
-# [START howto_operator_gcf_deploy_body]
-body = {
-    "name": FUNCTION_NAME,
-    "entryPoint": ENTRYPOINT,
-    "runtime": RUNTIME,
-    "httpsTrigger": {}
-}
-# [END howto_operator_gcf_deploy_body]
-
-# [START howto_operator_gcf_default_args]
-default_args = {
-    'start_date': dates.days_ago(1)
-}
-# [END howto_operator_gcf_default_args]
-
-# [START howto_operator_gcf_deploy_variants]
-if SOURCE_ARCHIVE_URL:
-    body['sourceArchiveUrl'] = SOURCE_ARCHIVE_URL
-elif SOURCE_REPOSITORY:
-    body['sourceRepository'] = {
-        'url': SOURCE_REPOSITORY
-    }
-elif ZIP_PATH:
-    body['sourceUploadUrl'] = ''
-    default_args['zip_path'] = ZIP_PATH
-elif SOURCE_UPLOAD_URL:
-    body['sourceUploadUrl'] = SOURCE_UPLOAD_URL
-else:
-    raise Exception("Please provide one of the source_code parameters")
-# [END howto_operator_gcf_deploy_variants]
-
-
-with models.DAG(
-    'example_gcp_function_deploy_delete',
-    default_args=default_args,
-    schedule_interval=datetime.timedelta(days=1)
-) as dag:
-    # [START howto_operator_gcf_deploy]
-    deploy_task = GcfFunctionDeployOperator(
-        task_id="gcf_deploy_task",
-        name=FUNCTION_NAME,
-        project_id=PROJECT_ID,
-        location=LOCATION,
-        body=body,
-        validate_body=VALIDATE_BODY
-    )
-    # [END howto_operator_gcf_deploy]
-    delete_task = GcfFunctionDeleteOperator(
-        task_id="gcf_delete_task",
-        name=FUNCTION_NAME
-    )
-    deploy_task >> delete_task
diff --git a/airflow/contrib/example_dags/example_gcp_spanner.py b/airflow/contrib/example_dags/example_gcp_spanner.py
index 0aeb1e63a0..936e5094e5 100644
--- a/airflow/contrib/example_dags/example_gcp_spanner.py
+++ b/airflow/contrib/example_dags/example_gcp_spanner.py
@@ -39,7 +39,8 @@
 import airflow
 from airflow import models
 from airflow.contrib.operators.gcp_spanner_operator import \
-    CloudSpannerInstanceDeployOperator, CloudSpannerInstanceDatabaseQueryOperator, \
+    CloudSpannerInstanceDeployOperator, \
+    CloudSpannerInstanceDatabaseQueryOperator, \
     CloudSpannerInstanceDeleteOperator, \
     CloudSpannerInstanceDatabaseDeployOperator, \
     CloudSpannerInstanceDatabaseUpdateOperator, \
@@ -76,17 +77,14 @@
         display_name=GCP_SPANNER_DISPLAY_NAME,
         task_id='spanner_instance_create_task'
     )
-    # [END howto_operator_spanner_deploy]
-
-    # Update
     spanner_instance_update_task = CloudSpannerInstanceDeployOperator(
-        project_id=GCP_PROJECT_ID,
         instance_id=GCP_SPANNER_INSTANCE_ID,
         configuration_name=GCP_SPANNER_CONFIG_NAME,
         node_count=int(GCP_SPANNER_NODE_COUNT) + 1,
         display_name=GCP_SPANNER_DISPLAY_NAME + '_updated',
         task_id='spanner_instance_update_task'
     )
+    # [END howto_operator_spanner_deploy]
 
     # [START howto_operator_spanner_database_deploy]
     spanner_database_deploy_task = CloudSpannerInstanceDatabaseDeployOperator(
@@ -99,6 +97,15 @@
         ],
         task_id='spanner_database_deploy_task'
     )
+    spanner_database_deploy_task2 = CloudSpannerInstanceDatabaseDeployOperator(
+        instance_id=GCP_SPANNER_INSTANCE_ID,
+        database_id=GCP_SPANNER_DATABASE_ID,
+        ddl_statements=[
+            "CREATE TABLE my_table1 (id INT64, name STRING(MAX)) PRIMARY KEY (id)",
+            "CREATE TABLE my_table2 (id INT64, name STRING(MAX)) PRIMARY KEY (id)",
+        ],
+        task_id='spanner_database_deploy_task2'
+    )
     # [END howto_operator_spanner_database_deploy]
 
     # [START howto_operator_spanner_database_update]
@@ -125,7 +132,6 @@
         task_id='spanner_database_update_idempotent1_task'
     )
     spanner_database_update_idempotent2_task = CloudSpannerInstanceDatabaseUpdateOperator(
-        project_id=GCP_PROJECT_ID,
         instance_id=GCP_SPANNER_INSTANCE_ID,
         database_id=GCP_SPANNER_DATABASE_ID,
         operation_id=OPERATION_ID,
@@ -140,19 +146,17 @@
     spanner_instance_query_task = CloudSpannerInstanceDatabaseQueryOperator(
         project_id=GCP_PROJECT_ID,
         instance_id=GCP_SPANNER_INSTANCE_ID,
-        database_id='db1',
+        database_id=GCP_SPANNER_DATABASE_ID,
         query=["DELETE FROM my_table2 WHERE true"],
-        task_id='spanner_instance_query'
+        task_id='spanner_instance_query_task'
     )
-    # [END howto_operator_spanner_query]
-
-    spanner_instance_query2_task = CloudSpannerInstanceDatabaseQueryOperator(
-        project_id=GCP_PROJECT_ID,
+    spanner_instance_query_task2 = CloudSpannerInstanceDatabaseQueryOperator(
         instance_id=GCP_SPANNER_INSTANCE_ID,
-        database_id='db1',
-        query="example_gcp_spanner.sql",
-        task_id='spanner_instance_query2'
+        database_id=GCP_SPANNER_DATABASE_ID,
+        query=["DELETE FROM my_table2 WHERE true"],
+        task_id='spanner_instance_query_task2'
     )
+    # [END howto_operator_spanner_query]
 
     # [START howto_operator_spanner_database_delete]
     spanner_database_delete_task = CloudSpannerInstanceDatabaseDeleteOperator(
@@ -161,6 +165,11 @@
         database_id=GCP_SPANNER_DATABASE_ID,
         task_id='spanner_database_delete_task'
     )
+    spanner_database_delete_task2 = CloudSpannerInstanceDatabaseDeleteOperator(
+        instance_id=GCP_SPANNER_INSTANCE_ID,
+        database_id=GCP_SPANNER_DATABASE_ID,
+        task_id='spanner_database_delete_task2'
+    )
     # [END howto_operator_spanner_database_delete]
 
     # [START howto_operator_spanner_delete]
@@ -169,15 +178,22 @@
         instance_id=GCP_SPANNER_INSTANCE_ID,
         task_id='spanner_instance_delete_task'
     )
+    spanner_instance_delete_task2 = CloudSpannerInstanceDeleteOperator(
+        instance_id=GCP_SPANNER_INSTANCE_ID,
+        task_id='spanner_instance_delete_task2'
+    )
     # [END howto_operator_spanner_delete]
 
     spanner_instance_create_task \
         >> spanner_instance_update_task \
         >> spanner_database_deploy_task \
+        >> spanner_database_deploy_task2 \
         >> spanner_database_update_task \
         >> spanner_database_update_idempotent1_task \
         >> spanner_database_update_idempotent2_task \
         >> spanner_instance_query_task \
-        >> spanner_instance_query2_task \
+        >> spanner_instance_query_task2 \
         >> spanner_database_delete_task \
-        >> spanner_instance_delete_task
+        >> spanner_database_delete_task2 \
+        >> spanner_instance_delete_task \
+        >> spanner_instance_delete_task2
diff --git a/airflow/contrib/example_dags/example_gcp_sql.py b/airflow/contrib/example_dags/example_gcp_sql.py
index c6838a2baf..1d0cb4d054 100644
--- a/airflow/contrib/example_dags/example_gcp_sql.py
+++ b/airflow/contrib/example_dags/example_gcp_sql.py
@@ -23,14 +23,13 @@
 
 This DAG relies on the following OS environment variables
 https://airflow.apache.org/concepts.html#variables
-* PROJECT_ID - Google Cloud Platform project for the Cloud SQL instance.
+* GCP_PROJECT_ID - Google Cloud Platform project for the Cloud SQL instance.
 * INSTANCE_NAME - Name of the Cloud SQL instance.
 * DB_NAME - Name of the database inside a Cloud SQL instance.
 """
 
 import os
 
-import re
 
 import airflow
 from airflow import models
@@ -43,16 +42,18 @@
     GoogleCloudStorageBucketCreateAclEntryOperator, \
     GoogleCloudStorageObjectCreateAclEntryOperator
 
+from six.moves.urllib.parse import urlsplit
+
 # [START howto_operator_cloudsql_arguments]
-PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project')
-INSTANCE_NAME = os.environ.get('INSTANCE_NAME', 'test-mysql')
-INSTANCE_NAME2 = os.environ.get('INSTANCE_NAME2', 'test-mysql2')
-DB_NAME = os.environ.get('DB_NAME', 'testdb')
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+INSTANCE_NAME = os.environ.get('GCSQL_MYSQL_INSTANCE_NAME', 'test-mysql')
+INSTANCE_NAME2 = os.environ.get('GCSQL_MYSQL_INSTANCE_NAME2', 'test-mysql2')
+DB_NAME = os.environ.get('GCSQL_MYSQL_DATABASE_NAME', 'testdb')
 # [END howto_operator_cloudsql_arguments]
 
 # [START howto_operator_cloudsql_export_import_arguments]
-EXPORT_URI = os.environ.get('EXPORT_URI', 'gs://bucketName/fileName')
-IMPORT_URI = os.environ.get('IMPORT_URI', 'gs://bucketName/fileName')
+EXPORT_URI = os.environ.get('GCSQL_MYSQL_EXPORT_URI', 'gs://bucketName/fileName')
+IMPORT_URI = os.environ.get('GCSQL_MYSQL_IMPORT_URI', 'gs://bucketName/fileName')
 # [END howto_operator_cloudsql_export_import_arguments]
 
 # Bodies below represent Cloud SQL instance resources:
@@ -145,7 +146,7 @@
 db_create_body = {
     "instance": INSTANCE_NAME,
     "name": DB_NAME,
-    "project": PROJECT_ID
+    "project": GCP_PROJECT_ID
 }
 # [END howto_operator_cloudsql_db_create_body]
 # [START howto_operator_cloudsql_db_patch_body]
@@ -162,9 +163,8 @@
 with models.DAG(
     'example_gcp_sql',
     default_args=default_args,
-    schedule_interval=None
+    schedule_interval=None  # Override to match your needs
 ) as dag:
-    prev_task = None
 
     def next_dep(task, prev):
         prev >> task
@@ -176,21 +176,20 @@ def next_dep(task, prev):
 
     # [START howto_operator_cloudsql_create]
     sql_instance_create = CloudSqlInstanceCreateOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         body=body,
         instance=INSTANCE_NAME,
-        task_id='sql_instance_create'
+        task_id='sql_instance_create_task'
     )
-    # [END howto_operator_cloudsql_create]
-    prev_task = sql_instance_create
-
-    sql_instance_create_2 = CloudSqlInstanceCreateOperator(
-        project_id=PROJECT_ID,
+    sql_instance_create_task2 = CloudSqlInstanceCreateOperator(
         body=body2,
         instance=INSTANCE_NAME2,
-        task_id='sql_instance_create_2'
+        task_id='sql_instance_create_task2'
     )
-    prev_task = next_dep(sql_instance_create_2, prev_task)
+    # [END howto_operator_cloudsql_create]
+
+    prev_task = sql_instance_create
+    prev_task = next_dep(sql_instance_create_task2, prev_task)
 
     # ############################################## #
     # ### MODIFYING INSTANCE AND ITS DATABASE ###### #
@@ -198,46 +197,69 @@ def next_dep(task, prev):
 
     # [START howto_operator_cloudsql_patch]
     sql_instance_patch_task = CloudSqlInstancePatchOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         body=patch_body,
         instance=INSTANCE_NAME,
         task_id='sql_instance_patch_task'
     )
+
+    sql_instance_patch_task2 = CloudSqlInstancePatchOperator(
+        body=patch_body,
+        instance=INSTANCE_NAME,
+        task_id='sql_instance_patch_task2'
+    )
     # [END howto_operator_cloudsql_patch]
     prev_task = next_dep(sql_instance_patch_task, prev_task)
+    prev_task = next_dep(sql_instance_patch_task2, prev_task)
 
     # [START howto_operator_cloudsql_db_create]
     sql_db_create_task = CloudSqlInstanceDatabaseCreateOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         body=db_create_body,
         instance=INSTANCE_NAME,
         task_id='sql_db_create_task'
     )
+    sql_db_create_task2 = CloudSqlInstanceDatabaseCreateOperator(
+        body=db_create_body,
+        instance=INSTANCE_NAME,
+        task_id='sql_db_create_task2'
+    )
     # [END howto_operator_cloudsql_db_create]
     prev_task = next_dep(sql_db_create_task, prev_task)
+    prev_task = next_dep(sql_db_create_task2, prev_task)
 
     # [START howto_operator_cloudsql_db_patch]
     sql_db_patch_task = CloudSqlInstanceDatabasePatchOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         body=db_patch_body,
         instance=INSTANCE_NAME,
         database=DB_NAME,
         task_id='sql_db_patch_task'
     )
+    sql_db_patch_task2 = CloudSqlInstanceDatabasePatchOperator(
+        body=db_patch_body,
+        instance=INSTANCE_NAME,
+        database=DB_NAME,
+        task_id='sql_db_patch_task2'
+    )
     # [END howto_operator_cloudsql_db_patch]
     prev_task = next_dep(sql_db_patch_task, prev_task)
+    prev_task = next_dep(sql_db_patch_task2, prev_task)
 
     # ############################################## #
     # ### EXPORTING SQL FROM INSTANCE 1 ############ #
     # ############################################## #
+    export_url_split = urlsplit(EXPORT_URI)
 
     # For export to work we need to add the Cloud SQL instance's Service Account
     # write access to the destination GCS bucket.
     # [START howto_operator_cloudsql_export_gcs_permissions]
     sql_gcp_add_bucket_permission = GoogleCloudStorageBucketCreateAclEntryOperator(
-        entity="user-{{ task_instance.xcom_pull('sql_instance_create', key='service_account_email') }}",
+        entity="user-{{ task_instance.xcom_pull("
+               "'sql_instance_create_task', key='service_account_email') "
+               "}}",
         role="WRITER",
-        bucket=re.match(r'gs:\/\/(\S*)\/', EXPORT_URI).group(1),
+        bucket=export_url_split[1],  # netloc (bucket)
         task_id='sql_gcp_add_bucket_permission'
     )
     # [END howto_operator_cloudsql_export_gcs_permissions]
@@ -245,40 +267,67 @@ def next_dep(task, prev):
 
     # [START howto_operator_cloudsql_export]
     sql_export_task = CloudSqlInstanceExportOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         body=export_body,
         instance=INSTANCE_NAME,
         task_id='sql_export_task'
     )
+    sql_export_task2 = CloudSqlInstanceExportOperator(
+        body=export_body,
+        instance=INSTANCE_NAME,
+        task_id='sql_export_task2'
+    )
     # [END howto_operator_cloudsql_export]
     prev_task = next_dep(sql_export_task, prev_task)
+    prev_task = next_dep(sql_export_task2, prev_task)
 
     # ############################################## #
     # ### IMPORTING SQL TO INSTANCE 2 ############## #
     # ############################################## #
+    import_url_split = urlsplit(IMPORT_URI)
 
     # For import to work we need to add the Cloud SQL instance's Service Account
     # read access to the target GCS object.
     # [START howto_operator_cloudsql_import_gcs_permissions]
     sql_gcp_add_object_permission = GoogleCloudStorageObjectCreateAclEntryOperator(
-        entity="user-{{ task_instance.xcom_pull('sql_instance_create_2', key='service_account_email') }}",
+        entity="user-{{ task_instance.xcom_pull("
+               "'sql_instance_create_task2', key='service_account_email')"
+               " }}",
         role="READER",
-        bucket=re.match(r'gs:\/\/(\S*)\/', IMPORT_URI).group(1),
-        object_name=re.match(r'gs:\/\/[^\/]*\/(\S*)', IMPORT_URI).group(1),
+        bucket=import_url_split[1],  # netloc (bucket)
+        object_name=import_url_split[2][1:],  # path (strip first '/')
         task_id='sql_gcp_add_object_permission',
     )
-    # [END howto_operator_cloudsql_import_gcs_permissions]
     prev_task = next_dep(sql_gcp_add_object_permission, prev_task)
 
+    # For import to work we also need to add the Cloud SQL instance's Service Account
+    # write access to the whole bucket!.
+    sql_gcp_add_bucket_permission2 = GoogleCloudStorageBucketCreateAclEntryOperator(
+        entity="user-{{ task_instance.xcom_pull("
+               "'sql_instance_create_task2', key='service_account_email') "
+               "}}",
+        role="WRITER",
+        bucket=import_url_split[1],  # netloc
+        task_id='sql_gcp_add_bucket_permission2',
+    )
+    # [END howto_operator_cloudsql_import_gcs_permissions]
+    prev_task = next_dep(sql_gcp_add_bucket_permission2, prev_task)
+
     # [START howto_operator_cloudsql_import]
     sql_import_task = CloudSqlInstanceImportOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         body=import_body,
         instance=INSTANCE_NAME2,
         task_id='sql_import_task'
     )
+    sql_import_task2 = CloudSqlInstanceImportOperator(
+        body=import_body,
+        instance=INSTANCE_NAME2,
+        task_id='sql_import_task2'
+    )
     # [END howto_operator_cloudsql_import]
     prev_task = next_dep(sql_import_task, prev_task)
+    prev_task = next_dep(sql_import_task2, prev_task)
 
     # ############################################## #
     # ### DELETING A DATABASE FROM AN INSTANCE ##### #
@@ -286,13 +335,19 @@ def next_dep(task, prev):
 
     # [START howto_operator_cloudsql_db_delete]
     sql_db_delete_task = CloudSqlInstanceDatabaseDeleteOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         instance=INSTANCE_NAME,
         database=DB_NAME,
         task_id='sql_db_delete_task'
     )
+    sql_db_delete_task2 = CloudSqlInstanceDatabaseDeleteOperator(
+        instance=INSTANCE_NAME,
+        database=DB_NAME,
+        task_id='sql_db_delete_task2'
+    )
     # [END howto_operator_cloudsql_db_delete]
     prev_task = next_dep(sql_db_delete_task, prev_task)
+    prev_task = next_dep(sql_db_delete_task2, prev_task)
 
     # ############################################## #
     # ### INSTANCES TEAR DOWN ###################### #
@@ -300,16 +355,14 @@ def next_dep(task, prev):
 
     # [START howto_operator_cloudsql_delete]
     sql_instance_delete_task = CloudSqlInstanceDeleteOperator(
-        project_id=PROJECT_ID,
+        project_id=GCP_PROJECT_ID,
         instance=INSTANCE_NAME,
         task_id='sql_instance_delete_task'
     )
-    # [END howto_operator_cloudsql_delete]
-    prev_task = next_dep(sql_instance_delete_task, prev_task)
-
-    sql_instance_delete_task_2 = CloudSqlInstanceDeleteOperator(
-        project_id=PROJECT_ID,
+    sql_instance_delete_task2 = CloudSqlInstanceDeleteOperator(
         instance=INSTANCE_NAME2,
-        task_id='sql_instance_delete_task_2'
+        task_id='sql_instance_delete_task2'
     )
-    prev_task = next_dep(sql_instance_delete_task_2, prev_task)
+    # [END howto_operator_cloudsql_delete]
+    prev_task = next_dep(sql_instance_delete_task, prev_task)
+    prev_task = next_dep(sql_instance_delete_task2, prev_task)
diff --git a/airflow/contrib/example_dags/example_gcp_sql_query.py b/airflow/contrib/example_dags/example_gcp_sql_query.py
index 5439fb6afc..f4fc5f3926 100644
--- a/airflow/contrib/example_dags/example_gcp_sql_query.py
+++ b/airflow/contrib/example_dags/example_gcp_sql_query.py
@@ -22,26 +22,25 @@
 
 This DAG relies on the following OS environment variables
 
-* PROJECT_ID - Google Cloud Platform project for the Cloud SQL instance
-* LOCATION - Google Cloud location where the database is created
+* GCP_PROJECT_ID - Google Cloud Platform project for the Cloud SQL instance
+* GCP_REGION - Google Cloud region where the database is created
 *
-* POSTGRES_INSTANCE_NAME - Name of the postgres Cloud SQL instance
-* POSTGRES_USER - Name of the postgres database user
-* POSTGRES_PASSWORD - Password of the postgres database user
-* POSTGRES_PROXY_PORT - Local port number for proxy connections for postgres
-* POSTGRES_PUBLIC_IP - Public IP of the Postgres database
-* POSTGRES_PUBLIC_PORT - Port of the postgres database
+* GCSQL_POSTGRES_INSTANCE_NAME - Name of the postgres Cloud SQL instance
+* GCSQL_POSTGRES_USER - Name of the postgres database user
+* GCSQL_POSTGRES_PASSWORD - Password of the postgres database user
+* GCSQL_POSTGRES_PUBLIC_IP - Public IP of the Postgres database
+* GCSQL_POSTGRES_PUBLIC_PORT - Port of the postgres database
 *
-* MYSQL_INSTANCE_NAME - Name of the postgres Cloud SQL instance
-* MYSQL_USER - Name of the mysql database user
-* MYSQL_PASSWORD - Password of the mysql database user
-* MYSQL_PROXY_PORT - Local port number for proxy connections for mysql
-* MYSQL_PUBLIC_IP - Public IP of the mysql database
-* MYSQL_PUBLIC_PORT - Port of the mysql database
+* GCSQL_MYSQL_INSTANCE_NAME - Name of the postgres Cloud SQL instance
+* GCSQL_MYSQL_USER - Name of the mysql database user
+* GCSQL_MYSQL_PASSWORD - Password of the mysql database user
+* GCSQL_MYSQL_PUBLIC_IP - Public IP of the mysql database
+* GCSQL_MYSQL_PUBLIC_PORT - Port of the mysql database
 """
 
 import os
 import subprocess
+from os.path import expanduser
 
 from six.moves.urllib.parse import quote_plus
 
@@ -51,34 +50,38 @@
 
 # [START howto_operator_cloudsql_query_arguments]
 
-PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project')
-LOCATION = os.environ.get('REGION', 'europe-west-1')
-
-POSTGRES_INSTANCE_NAME = os.environ.get('POSTGRES_INSTANCE_NAME', 'testpostgres')
-POSTGRES_DATABASE_NAME = os.environ.get('POSTGRES_DATABASE_NAME', 'postgresdb')
-POSTGRES_USER = os.environ.get('POSTGRES_USER', 'postgres_user')
-POSTGRES_PASSWORD = os.environ.get('POSTGRES_PASSWORD', 'password')
-POSTGRES_PUBLIC_IP = os.environ.get('POSTGRES_PUBLIC_IP', '0.0.0.0')
-POSTGRES_PUBLIC_PORT = os.environ.get('POSTGRES_PUBLIC_PORT', 5432)
-POSTGRES_CLIENT_CERT_FILE = os.environ.get('POSTGRES_CLIENT_CERT_FILE',
-                                           "/tmp/client-cert.pem")
-POSTGRES_CLIENT_KEY_FILE = os.environ.get('POSTGRES_CLIENT_KEY_FILE',
-                                          "/tmp/client-key.pem")
-POSTGRES_SERVER_CA_FILE = os.environ.get('POSTGRES_SERVER_CA_FILE',
-                                         "/tmp/server-ca.pem")
-
-MYSQL_INSTANCE_NAME = os.environ.get('MYSQL_INSTANCE_NAME', 'testmysql')
-MYSQL_DATABASE_NAME = os.environ.get('MYSQL_DATABASE_NAME', 'mysqldb')
-MYSQL_USER = os.environ.get('MYSQL_USER', 'mysql_user')
-MYSQL_PASSWORD = os.environ.get('MYSQL_PASSWORD', 'password')
-MYSQL_PUBLIC_IP = os.environ.get('MYSQL_PUBLIC_IP', '0.0.0.0')
-MYSQL_PUBLIC_PORT = os.environ.get('MYSQL_PUBLIC_PORT', 3306)
-MYSQL_CLIENT_CERT_FILE = os.environ.get('MYSQL_CLIENT_CERT_FILE',
-                                        "/tmp/client-cert.pem")
-MYSQL_CLIENT_KEY_FILE = os.environ.get('MYSQL_CLIENT_KEY_FILE',
-                                       "/tmp/client-key.pem")
-MYSQL_SERVER_CA_FILE = os.environ.get('MYSQL_SERVER_CA_FILE',
-                                      "/tmp/server-ca.pem")
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCP_REGION = os.environ.get('GCP_REGION', 'europe-west-1b')
+
+GCSQL_POSTGRES_INSTANCE_NAME_QUERY = os.environ.get(
+    'GCSQL_POSTGRES_INSTANCE_NAME_QUERY',
+    'testpostgres')
+GCSQL_POSTGRES_DATABASE_NAME = os.environ.get('GCSQL_POSTGRES_DATABASE_NAME',
+                                              'postgresdb')
+GCSQL_POSTGRES_USER = os.environ.get('GCSQL_POSTGRES_USER', 'postgres_user')
+GCSQL_POSTGRES_PASSWORD = os.environ.get('GCSQL_POSTGRES_PASSWORD', 'password')
+GCSQL_POSTGRES_PUBLIC_IP = os.environ.get('GCSQL_POSTGRES_PUBLIC_IP', '0.0.0.0')
+GCSQL_POSTGRES_PUBLIC_PORT = os.environ.get('GCSQL_POSTGRES_PUBLIC_PORT', 5432)
+GCSQL_POSTGRES_CLIENT_CERT_FILE = os.environ.get('GCSQL_POSTGRES_CLIENT_CERT_FILE',
+                                                 ".key/postgres-client-cert.pem")
+GCSQL_POSTGRES_CLIENT_KEY_FILE = os.environ.get('GCSQL_POSTGRES_CLIENT_KEY_FILE',
+                                                ".key/postgres-client-key.pem")
+GCSQL_POSTGRES_SERVER_CA_FILE = os.environ.get('GCSQL_POSTGRES_SERVER_CA_FILE',
+                                               ".key/postgres-server-ca.pem")
+
+GCSQL_MYSQL_INSTANCE_NAME_QUERY = os.environ.get('GCSQL_MYSQL_INSTANCE_NAME_QUERY',
+                                                 'testmysql')
+GCSQL_MYSQL_DATABASE_NAME = os.environ.get('GCSQL_MYSQL_DATABASE_NAME', 'mysqldb')
+GCSQL_MYSQL_USER = os.environ.get('GCSQL_MYSQL_USER', 'mysql_user')
+GCSQL_MYSQL_PASSWORD = os.environ.get('GCSQL_MYSQL_PASSWORD', 'password')
+GCSQL_MYSQL_PUBLIC_IP = os.environ.get('GCSQL_MYSQL_PUBLIC_IP', '0.0.0.0')
+GCSQL_MYSQL_PUBLIC_PORT = os.environ.get('GCSQL_MYSQL_PUBLIC_PORT', 3306)
+GCSQL_MYSQL_CLIENT_CERT_FILE = os.environ.get('GCSQL_MYSQL_CLIENT_CERT_FILE',
+                                              ".key/mysql-client-cert.pem")
+GCSQL_MYSQL_CLIENT_KEY_FILE = os.environ.get('GCSQL_MYSQL_CLIENT_KEY_FILE',
+                                             ".key/mysql-client-key.pem")
+GCSQL_MYSQL_SERVER_CA_FILE = os.environ.get('GCSQL_MYSQL_SERVER_CA_FILE',
+                                            ".key/mysql-server-ca.pem")
 
 SQL = [
     'CREATE TABLE IF NOT EXISTS TABLE_TEST (I INTEGER)',
@@ -97,18 +100,28 @@
 
 # [START howto_operator_cloudsql_query_connections]
 
+HOME_DIR = expanduser("~")
+
+
+def get_absolute_path(path):
+    if path.startswith("/"):
+        return path
+    else:
+        return os.path.join(HOME_DIR, path)
+
+
 postgres_kwargs = dict(
-    user=quote_plus(POSTGRES_USER),
-    password=quote_plus(POSTGRES_PASSWORD),
-    public_port=POSTGRES_PUBLIC_PORT,
-    public_ip=quote_plus(POSTGRES_PUBLIC_IP),
-    project_id=quote_plus(PROJECT_ID),
-    location=quote_plus(LOCATION),
-    instance=quote_plus(POSTGRES_INSTANCE_NAME),
-    database=quote_plus(POSTGRES_DATABASE_NAME),
-    client_cert_file=quote_plus(POSTGRES_CLIENT_CERT_FILE),
-    client_key_file=quote_plus(POSTGRES_CLIENT_KEY_FILE),
-    server_ca_file=quote_plus(POSTGRES_SERVER_CA_FILE)
+    user=quote_plus(GCSQL_POSTGRES_USER),
+    password=quote_plus(GCSQL_POSTGRES_PASSWORD),
+    public_port=GCSQL_POSTGRES_PUBLIC_PORT,
+    public_ip=quote_plus(GCSQL_POSTGRES_PUBLIC_IP),
+    project_id=quote_plus(GCP_PROJECT_ID),
+    location=quote_plus(GCP_REGION),
+    instance=quote_plus(GCSQL_POSTGRES_INSTANCE_NAME_QUERY),
+    database=quote_plus(GCSQL_POSTGRES_DATABASE_NAME),
+    client_cert_file=quote_plus(get_absolute_path(GCSQL_POSTGRES_CLIENT_CERT_FILE)),
+    client_key_file=quote_plus(get_absolute_path(GCSQL_POSTGRES_CLIENT_KEY_FILE)),
+    server_ca_file=quote_plus(get_absolute_path(GCSQL_POSTGRES_SERVER_CA_FILE))
 )
 
 # The connections below are created using one of the standard approaches - via environment
@@ -161,17 +174,17 @@
     .format(**postgres_kwargs)
 
 mysql_kwargs = dict(
-    user=quote_plus(MYSQL_USER),
-    password=quote_plus(MYSQL_PASSWORD),
-    public_port=MYSQL_PUBLIC_PORT,
-    public_ip=quote_plus(MYSQL_PUBLIC_IP),
-    project_id=quote_plus(PROJECT_ID),
-    location=quote_plus(LOCATION),
-    instance=quote_plus(MYSQL_INSTANCE_NAME),
-    database=quote_plus(MYSQL_DATABASE_NAME),
-    client_cert_file=quote_plus(MYSQL_CLIENT_CERT_FILE),
-    client_key_file=quote_plus(MYSQL_CLIENT_KEY_FILE),
-    server_ca_file=quote_plus(MYSQL_SERVER_CA_FILE)
+    user=quote_plus(GCSQL_MYSQL_USER),
+    password=quote_plus(GCSQL_MYSQL_PASSWORD),
+    public_port=GCSQL_MYSQL_PUBLIC_PORT,
+    public_ip=quote_plus(GCSQL_MYSQL_PUBLIC_IP),
+    project_id=quote_plus(GCP_PROJECT_ID),
+    location=quote_plus(GCP_REGION),
+    instance=quote_plus(GCSQL_MYSQL_INSTANCE_NAME_QUERY),
+    database=quote_plus(GCSQL_MYSQL_DATABASE_NAME),
+    client_cert_file=quote_plus(get_absolute_path(GCSQL_MYSQL_CLIENT_CERT_FILE)),
+    client_key_file=quote_plus(get_absolute_path(GCSQL_MYSQL_CLIENT_KEY_FILE)),
+    server_ca_file=quote_plus(get_absolute_path(GCSQL_MYSQL_SERVER_CA_FILE))
 )
 
 # MySQL: connect via proxy over TCP (specific proxy version)
@@ -226,6 +239,21 @@
     "sslkey={client_key_file}&" \
     "sslrootcert={server_ca_file}".format(**mysql_kwargs)
 
+# Special case: MySQL: connect directly via TCP (SSL) and with fixed Cloud Sql
+# Proxy binary path AND with missing project_id
+
+os.environ['AIRFLOW_CONN_PUBLIC_MYSQL_TCP_SSL_NO_PROJECT_ID'] = \
+    "gcpcloudsql://{user}:{password}@{public_ip}:{public_port}/{database}?" \
+    "database_type=mysql&" \
+    "location={location}&" \
+    "instance={instance}&" \
+    "use_proxy=False&" \
+    "use_ssl=True&" \
+    "sslcert={client_cert_file}&" \
+    "sslkey={client_key_file}&" \
+    "sslrootcert={server_ca_file}".format(**mysql_kwargs)
+
+
 # [END howto_operator_cloudsql_query_connections]
 
 # [START howto_operator_cloudsql_query_operators]
@@ -238,22 +266,29 @@
     "proxy_mysql_tcp",
     "proxy_mysql_socket",
     "public_mysql_tcp",
-    "public_mysql_tcp_ssl"
+    "public_mysql_tcp_ssl",
+    "public_mysql_tcp_ssl_no_project_id"
 ]
 
 tasks = []
 
+
 with models.DAG(
     dag_id='example_gcp_sql_query',
     default_args=default_args,
     schedule_interval=None
 ) as dag:
+    prev_task = None
+
     for connection_name in connection_names:
-        tasks.append(
-            CloudSqlQueryOperator(
-                gcp_cloudsql_conn_id=connection_name,
-                task_id="example_gcp_sql_task_" + connection_name,
-                sql=SQL
-            )
+        task = CloudSqlQueryOperator(
+            gcp_cloudsql_conn_id=connection_name,
+            task_id="example_gcp_sql_task_" + connection_name,
+            sql=SQL
         )
+        tasks.append(task)
+        if prev_task:
+            prev_task >> task
+        prev_task = task
+
 # [END howto_operator_cloudsql_query_operators]
diff --git a/airflow/contrib/hooks/gcp_api_base_hook.py b/airflow/contrib/hooks/gcp_api_base_hook.py
index 696f00da6a..364a2f13ed 100644
--- a/airflow/contrib/hooks/gcp_api_base_hook.py
+++ b/airflow/contrib/hooks/gcp_api_base_hook.py
@@ -18,6 +18,7 @@
 # under the License.
 #
 import json
+import functools
 
 import httplib2
 import google.auth
@@ -142,7 +143,7 @@ def _get_field(self, f, default=None):
         key_path, etc. They get formatted as shown below.
         """
         long_f = 'extra__google_cloud_platform__{}'.format(f)
-        if long_f in self.extras:
+        if hasattr(self, 'extras') and long_f in self.extras:
             return self.extras[long_f]
         else:
             return default
@@ -150,3 +151,38 @@ def _get_field(self, f, default=None):
     @property
     def project_id(self):
         return self._get_field('project')
+
+    def fallback_to_default_project_id(func):
+        """
+        Decorator that provides fallback for Google Cloud Platform project id. If
+        the project is None it will be replaced with the project_id from the
+        service account the Hook is authenticated with. Project id can be specified
+        either via project_id kwarg or via first parameter in positional args.
+        :param func: function to wrap
+        :return: result of the function call
+        """
+        @functools.wraps(func)
+        def inner_wrapper(self, *args, **kwargs):
+            if 'project_id' in kwargs:
+                kwargs['project_id'] = self._get_project_id(kwargs['project_id'])
+            else:
+                # assume project_id is the first non-keyword parameter if
+                # no project_id in keyword params
+                largs = list(args)
+                largs[0] = self._get_project_id(largs[0])
+                args = tuple(largs)
+                self.log.warning("%s", args)
+            return func(self, *args, **kwargs)
+        return inner_wrapper
+
+    fallback_to_default_project_id = staticmethod(fallback_to_default_project_id)
+
+    def _get_project_id(self, project_id):
+        """
+        In case project_id is None, overrides it with default project_id from
+        the service account that is authorized.
+        :param project_id: project id to
+        :type project_id: str
+        :return: the project_id specified or default project id if project_id is None
+        """
+        return project_id if project_id else self.project_id
diff --git a/airflow/contrib/hooks/gcp_bigtable_hook.py b/airflow/contrib/hooks/gcp_bigtable_hook.py
index 5d1b6f01c9..a6b201728d 100644
--- a/airflow/contrib/hooks/gcp_bigtable_hook.py
+++ b/airflow/contrib/hooks/gcp_bigtable_hook.py
@@ -37,17 +37,21 @@ def __init__(self,
                  delegate_to=None):
         super(BigtableHook, self).__init__(gcp_conn_id, delegate_to)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_client(self, project_id):
         if not self._client:
-            self._client = Client(project=project_id, credentials=self._get_credentials(), admin=True)
+            self._client = Client(project=project_id, credentials=self._get_credentials(),
+                                  admin=True)
         return self._client
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_instance(self, project_id, instance_id):
         """
         Retrieves and returns the specified Cloud Bigtable instance if it exists.
         Otherwise, returns None.
 
-        :param project_id: The ID of the GCP project.
+        :param project_id: The ID of the GCP project. If set to None, default project_id
+            from service account is used.
         :type project_id: str
         :param instance_id: The ID of the Cloud Bigtable instance.
         :type instance_id: str
@@ -60,12 +64,15 @@ def get_instance(self, project_id, instance_id):
             return None
         return instance
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def delete_instance(self, project_id, instance_id):
         """
         Deletes the specified Cloud Bigtable instance.
-        Raises google.api_core.exceptions.NotFound if the Cloud Bigtable instance does not exist.
+        Raises google.api_core.exceptions.NotFound if the Cloud Bigtable instance does
+        not exist.
 
-        :param project_id: The ID of the GCP project.
+        :param project_id: The ID of the GCP project. If set to None, default project_id
+            from service account is used.
         :type project_id: str
         :param instance_id: The ID of the Cloud Bigtable instance.
         :type instance_id: str
@@ -73,6 +80,7 @@ def delete_instance(self, project_id, instance_id):
         instance = Instance(instance_id, self.get_client(project_id))
         instance.delete()
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def create_instance(self,
                         project_id,
                         instance_id,
@@ -90,7 +98,8 @@ def create_instance(self,
         Creates new instance.
 
         :type project_id: str
-        :param project_id: The ID of the GCP project.
+        :param project_id: The ID of the GCP project. If set to None, default project_id
+            from service account is used.
         :type instance_id: str
         :param instance_id: The ID for the new instance.
         :type main_cluster_id: str
@@ -99,7 +108,8 @@ def create_instance(self,
         :param main_cluster_zone: The zone for main cluster.
             See https://cloud.google.com/bigtable/docs/locations for more details.
         :type replica_cluster_id: str
-        :param replica_cluster_id: (optional) The ID for replica cluster for the new instance.
+        :param replica_cluster_id: (optional) The ID for replica cluster for the new
+            instance.
         :type replica_cluster_zone: str
         :param replica_cluster_zone: (optional)  The zone for replica cluster.
         :type instance_type: enums.Instance.Type
@@ -108,7 +118,8 @@ def create_instance(self,
         :param instance_display_name: (optional) Human-readable name of the instance.
                 Defaults to ``instance_id``.
         :type instance_labels: dict
-        :param instance_labels: (optional) Dictionary of labels to associate with the instance.
+        :param instance_labels: (optional) Dictionary of labels to associate with the
+            instance.
         :type cluster_nodes: int
         :param cluster_nodes: (optional) Number of nodes for cluster.
         :type cluster_storage_type: enums.StorageType
@@ -160,21 +171,25 @@ def create_table(self, instance, table_id, initial_split_keys, column_families):
         :type table_id: str
         :param table_id: The ID of the table to create in Cloud Bigtable.
         :type initial_split_keys: list
-        :param initial_split_keys: (Optional) A list of row keys in bytes to use to initially split the table.
+        :param initial_split_keys: (Optional) A list of row keys in bytes to use to
+            initially split the table.
         :type column_families: dict
-        :param column_families: (Optional) A map of columns to create. The key is the column_id str, and the
+        :param column_families: (Optional) A map of columns to create. The key is the
+            column_id str, and the
         value is a GarbageCollectionRule.
         """
         table = Table(table_id, instance)
         table.create(initial_split_keys, column_families)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def delete_table(self, project_id, instance_id, table_id):
         """
         Deletes the specified table in Cloud Bigtable.
         Raises google.api_core.exceptions.NotFound if the table does not exist.
 
         :type project_id: str
-        :param project_id: The ID of the GCP project.
+        :param project_id: The ID of the GCP project. If set to None, default project_id
+            from service account is used.
         :type instance_id: str
         :param instance_id: The ID of the Cloud Bigtable instance.
         :type table_id: str
@@ -209,7 +224,8 @@ def get_column_families_for_table(instance, table_id):
         :type instance: Instance
         :param instance: The Cloud Bigtable instance that owns the table.
         :type table_id: str
-        :param table_id: The ID of the table in Cloud Bigtable to fetch Column Families from.
+        :param table_id: The ID of the table in Cloud Bigtable to fetch Column Families
+            from.
         """
 
         table = Table(table_id, instance)
@@ -224,7 +240,8 @@ def get_cluster_states_for_table(instance, table_id):
         :type instance: Instance
         :param instance: The Cloud Bigtable instance that owns the table.
         :type table_id: str
-        :param table_id: The ID of the table in Cloud Bigtable to fetch Cluster States from.
+        :param table_id: The ID of the table in Cloud Bigtable to fetch Cluster States
+            from.
         """
 
         table = Table(table_id, instance)
diff --git a/airflow/contrib/hooks/gcp_compute_hook.py b/airflow/contrib/hooks/gcp_compute_hook.py
index 617e39cb40..cad8ce4eec 100644
--- a/airflow/contrib/hooks/gcp_compute_hook.py
+++ b/airflow/contrib/hooks/gcp_compute_hook.py
@@ -64,12 +64,14 @@ def get_conn(self):
                                http=http_authorized, cache_discovery=False)
         return self._conn
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def start_instance(self, project_id, zone, resource_id):
         """
         Starts an existing instance defined by project_id, zone and resource_id.
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance exists
+            Instance exists. if set to None, default project_id is used from
+            service account
         :type project_id: str
         :param zone: Google Cloud Platform zone where the instance exists
         :type zone: str
@@ -91,12 +93,14 @@ def start_instance(self, project_id, zone, resource_id):
                 "'name' field".format(response))
         return self._wait_for_operation_to_complete(project_id, operation_name, zone)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def stop_instance(self, project_id, zone, resource_id):
         """
         Stops an instance defined by project_id, zone and resource_id
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance exists
+            Instance exists. if set to None, default project_id is used from
+            service account
         :type project_id: str
         :param zone: Google Cloud Platform zone where the instance exists
         :type zone: str
@@ -118,12 +122,14 @@ def stop_instance(self, project_id, zone, resource_id):
                 "'name' field".format(response))
         return self._wait_for_operation_to_complete(project_id, operation_name, zone)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def set_machine_type(self, project_id, zone, resource_id, body):
         """
         Sets machine type of an instance defined by project_id, zone and resource_id.
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance exists
+            Instance exists. if set to None, default project_id is used from
+            service account
         :type project_id: str
         :param zone: Google Cloud Platform zone where the instance exists.
         :type zone: str
@@ -150,12 +156,14 @@ def _execute_set_machine_type(self, project_id, zone, resource_id, body):
             project=project_id, zone=zone, instance=resource_id, body=body)\
             .execute(num_retries=NUM_RETRIES)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_instance_template(self, project_id, resource_id):
         """
         Retrieves instance template by project_id and resource_id.
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance template exists
+            Instance exists. if set to None, default project_id is used from
+            service account
         :type project_id: str
         :param resource_id: Name of the instance template
         :type resource_id: str
@@ -169,12 +177,14 @@ def get_instance_template(self, project_id, resource_id):
         ).execute(num_retries=NUM_RETRIES)
         return response
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def insert_instance_template(self, project_id, body, request_id=None):
         """
         Inserts instance template using body specified
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance exists
+            Instance exists. if set to None, default project_id is used from
+            service account
         :type project_id: str
         :param body: Instance template representation as object according to
             https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates
@@ -200,12 +210,14 @@ def insert_instance_template(self, project_id, body, request_id=None):
                 "'name' field".format(response))
         return self._wait_for_operation_to_complete(project_id, operation_name)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_instance_group_manager(self, project_id, zone, resource_id):
         """
         Retrieves Instance Group Manager by project_id, zone and resource_id.
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance Group Manager exists
+            Instance Group Manager exists. if set to None, default project_id is
+            used from service account
         :type project_id: str
         :param zone: Google Cloud Platform zone where the Instance Group Manager exists
         :type zone: str
@@ -222,13 +234,15 @@ def get_instance_group_manager(self, project_id, zone, resource_id):
         ).execute(num_retries=NUM_RETRIES)
         return response
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def patch_instance_group_manager(self, project_id, zone, resource_id,
                                      body, request_id=None):
         """
         Patches Instance Group Manager with the specified body.
 
         :param project_id: Google Cloud Platform project ID where the Compute Engine
-            Instance Group Manager exists
+            Instance Group Manager exists. if set to None, default project_id is
+            used from service account
         :type project_id: str
         :param zone: Google Cloud Platform zone where the Instance Group Manager exists
         :type zone: str
diff --git a/airflow/contrib/hooks/gcp_function_hook.py b/airflow/contrib/hooks/gcp_function_hook.py
index 29cef1716c..3f1bacb45c 100644
--- a/airflow/contrib/hooks/gcp_function_hook.py
+++ b/airflow/contrib/hooks/gcp_function_hook.py
@@ -46,11 +46,25 @@ def __init__(self,
         super(GcfHook, self).__init__(gcp_conn_id, delegate_to)
         self.api_version = api_version
 
+    @staticmethod
+    def _full_location(project_id, location):
+        """
+        Retrieve full location of the function in the form of
+        projects/<GCP_PROJECT_ID>/locations/<GCP_LOCATION>
+        :param project_id: The Google Cloud Project project_id where the function belongs.
+            If set to None, default project_id is used from service account.
+        :type project_id: str
+        :param location: The location where the function is created.
+        :type location: str
+        :return:
+        """
+        return 'projects/{}/locations/{}'.format(project_id, location)
+
     def get_conn(self):
         """
         Retrieves the connection to Cloud Functions.
 
-        :return: Google Cloud Function services object
+        :return: Google Cloud Function services object.
         :rtype: dict
         """
         if not self._conn:
@@ -63,42 +77,31 @@ def get_function(self, name):
         """
         Returns the Cloud Function with the given name.
 
-        :param name: name of the function
+        :param name: Name of the function.
         :type name: str
-        :return: a Cloud Functions object representing the function
+        :return: A Cloud Functions object representing the function.
         :rtype: dict
         """
         return self.get_conn().projects().locations().functions().get(
             name=name).execute(num_retries=NUM_RETRIES)
 
-    def list_functions(self, full_location):
-        """
-        Lists all Cloud Functions created in the location.
-
-        :param full_location: full location including the project in the form of
-            of /projects/<PROJECT>/location/<LOCATION>
-        :type full_location: str
-        :return: array of Cloud Functions objects - representing functions in the location
-        :rtype: [dict]
-        """
-        list_response = self.get_conn().projects().locations().functions().list(
-            parent=full_location).execute(num_retries=NUM_RETRIES)
-        return list_response.get("functions", [])
-
-    def create_new_function(self, full_location, body):
+    @GoogleCloudBaseHook.fallback_to_default_project_id
+    def create_new_function(self, project_id, location, body):
         """
         Creates a new function in Cloud Function in the location specified in the body.
 
-        :param full_location: full location including the project in the form of
-            of /projects/<PROJECT>/location/<LOCATION>
-        :type full_location: str
-        :param body: body required by the Cloud Functions insert API
+        :param project_id: The Google Cloud Project project_id where the function belongs.
+            If set to None, default project_id is used from service account.
+        :type project_id: str
+        :param location: The location of the function.
+        :type location: str
+        :param body: The body required by the Cloud Functions insert API.
         :type body: dict
-        :return: response returned by the operation
+        :return: The response returned by the operation.
         :rtype: dict
         """
         response = self.get_conn().projects().locations().functions().create(
-            location=full_location,
+            location=self._full_location(project_id, location),
             body=body
         ).execute(num_retries=NUM_RETRIES)
         operation_name = response["name"]
@@ -108,13 +111,13 @@ def update_function(self, name, body, update_mask):
         """
         Updates Cloud Functions according to the specified update mask.
 
-        :param name: name of the function
+        :param name: The name of the function.
         :type name: str
-        :param body: body required by the cloud function patch API
+        :param body: The body required by the cloud function patch API.
         :type body: str
-        :param update_mask: update mask - array of fields that should be patched
+        :param update_mask: The update mask - array of fields that should be patched.
         :type update_mask: [str]
-        :return: response returned by the operation
+        :return: The response returned by the operation.
         :rtype: dict
         """
         response = self.get_conn().projects().locations().functions().patch(
@@ -125,25 +128,28 @@ def update_function(self, name, body, update_mask):
         operation_name = response["name"]
         return self._wait_for_operation_to_complete(operation_name)
 
-    def upload_function_zip(self, parent, zip_path):
+    @GoogleCloudBaseHook.fallback_to_default_project_id
+    def upload_function_zip(self, project_id, location, zip_path):
         """
         Uploads zip file with sources.
 
-        :param parent: Google Cloud Platform project id and region where zip file should
-         be uploaded in the form of /projects/<PROJECT>/location/<LOCATION>
-        :type parent: str
-        :param zip_path: path of the valid .zip file to upload
+        :param project_id: The Google Cloud Project project_id where the function belongs.
+            If set to None, default project_id is used from service account.
+        :type project_id: str
+        :param location: The location where the function is created.
+        :type location: str
+        :param zip_path: The path of the valid .zip file to upload.
         :type zip_path: str
-        :return: Upload URL that was returned by generateUploadUrl method
+        :return: The upload URL that was returned by generateUploadUrl method.
         """
         response = self.get_conn().projects().locations().functions().generateUploadUrl(
-            parent=parent
+            parent=self._full_location(project_id, location)
         ).execute(num_retries=NUM_RETRIES)
         upload_url = response.get('uploadUrl')
         with open(zip_path, 'rb') as fp:
             requests.put(
                 url=upload_url,
-                data=fp.read(),
+                data=fp,
                 # Those two headers needs to be specified according to:
                 # https://cloud.google.com/functions/docs/reference/rest/v1/projects.locations.functions/generateUploadUrl
                 # nopep8
@@ -158,9 +164,9 @@ def delete_function(self, name):
         """
         Deletes the specified Cloud Function.
 
-        :param name: name of the function
+        :param name: The name of the function.
         :type name: str
-        :return: response returned by the operation
+        :return: The response returned by the operation.
         :rtype: dict
         """
         response = self.get_conn().projects().locations().functions().delete(
@@ -173,11 +179,11 @@ def _wait_for_operation_to_complete(self, operation_name):
         Waits for the named operation to complete - checks status of the
         asynchronous call.
 
-        :param operation_name: name of the operation
+        :param operation_name: The name of the operation.
         :type operation_name: str
-        :return: response  returned by the operation
+        :return: The response returned by the operation.
         :rtype: dict
-        :exception: AirflowException in case error is returned
+        :exception: AirflowException in case error is returned.
         """
         service = self.get_conn()
         while True:
diff --git a/airflow/contrib/hooks/gcp_spanner_hook.py b/airflow/contrib/hooks/gcp_spanner_hook.py
index 23f3a3c86d..e4b077cd6d 100644
--- a/airflow/contrib/hooks/gcp_spanner_hook.py
+++ b/airflow/contrib/hooks/gcp_spanner_hook.py
@@ -19,9 +19,7 @@
 from google.api_core.exceptions import GoogleAPICallError, AlreadyExists
 from google.cloud.spanner_v1.client import Client
 from google.cloud.spanner_v1.database import Database
-from google.cloud.spanner_v1.instance import Instance  # noqa: F401
 from google.longrunning.operations_grpc_pb2 import Operation  # noqa: F401
-from typing import Optional, Callable  # noqa: F401
 
 from airflow import AirflowException
 from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
@@ -39,13 +37,14 @@ def __init__(self,
                  delegate_to=None):
         super(CloudSpannerHook, self).__init__(gcp_conn_id, delegate_to)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_client(self, project_id):
-        # type: (str) -> Client
+        # noinspection LongLine
         """
         Provides a client for interacting with the Cloud Spanner API.
 
         :param project_id: The ID of the  GCP project that owns the Cloud Spanner
-            database.
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :return: Client for interacting with the Cloud Spanner API. See:
             https://googleapis.github.io/google-cloud-python/latest/spanner/client-api.html#google.cloud.spanner_v1.client.Client
@@ -55,12 +54,14 @@ def get_client(self, project_id):
             self._client = Client(project=project_id, credentials=self._get_credentials())
         return self._client
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_instance(self, project_id, instance_id):
-        # type: (str, str) -> Optional[Instance]
+        # noinspection LongLine
         """
         Gets information about a particular instance.
 
-        :param project_id: The ID of the project which owns the Cloud Spanner Database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -68,18 +69,21 @@ def get_instance(self, project_id, instance_id):
             https://googleapis.github.io/google-cloud-python/latest/spanner/instance-api.html#google.cloud.spanner_v1.instance.Instance
         :rtype: object
         """
+        # noinspection PyUnresolvedReferences
         instance = self.get_client(project_id).instance(instance_id)
         if not instance.exists():
             return None
         return instance
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def create_instance(self, project_id, instance_id, configuration_name, node_count,
                         display_name):
-        # type: (str, str, str, int, str) -> bool
+        # noinspection LongLine
         """
         Creates a new Cloud Spanner instance.
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -100,13 +104,15 @@ def create_instance(self, project_id, instance_id, configuration_name, node_coun
         return self._apply_to_instance(project_id, instance_id, configuration_name,
                                        node_count, display_name, lambda x: x.create())
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def update_instance(self, project_id, instance_id, configuration_name, node_count,
                         display_name):
-        # type: (str, str, str, int, str) -> bool
+        # noinspection LongLine
         """
         Updates an existing Cloud Spanner instance.
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -129,11 +135,11 @@ def update_instance(self, project_id, instance_id, configuration_name, node_coun
 
     def _apply_to_instance(self, project_id, instance_id, configuration_name, node_count,
                            display_name, func):
-        # type: (str, str, str, int, str, Callable[[Instance], Operation]) -> bool
         """
         Invokes a method on a given instance by applying a specified Callable.
 
-        :param project_id: The ID of the project which owns the Cloud Spanner Database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the instance.
         :type instance_id: str
@@ -149,6 +155,7 @@ def _apply_to_instance(self, project_id, instance_id, configuration_name, node_c
         :param func: Method of the instance to be called.
         :type func: Callable
         """
+        # noinspection PyUnresolvedReferences
         instance = self.get_client(project_id).instance(
             instance_id, configuration_name=configuration_name,
             node_count=node_count, display_name=display_name)
@@ -163,12 +170,13 @@ def _apply_to_instance(self, project_id, instance_id, configuration_name, node_c
             self.log.info(result)
         return True
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def delete_instance(self, project_id, instance_id):
-        # type: (str, str) -> bool
         """
         Deletes an existing Cloud Spanner instance.
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id:  The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -181,13 +189,14 @@ def delete_instance(self, project_id, instance_id):
             self.log.error('An error occurred: %s. Exiting.', e.message)
             raise e
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_database(self, project_id, instance_id, database_id):
-        # type: (str, str, str) -> Optional[Database]
         """
         Retrieves a database in Cloud Spanner. If the database does not exist
         in the specified instance, it returns None.
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -208,12 +217,14 @@ def get_database(self, project_id, instance_id, database_id):
         else:
             return database
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def create_database(self, project_id, instance_id, database_id, ddl_statements):
         # type: (str, str, str, [str]) -> bool
         """
         Creates a new database in Cloud Spanner.
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -243,13 +254,14 @@ def create_database(self, project_id, instance_id, database_id, ddl_statements):
             self.log.info(result)
         return True
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def update_database(self, project_id, instance_id, database_id, ddl_statements,
                         operation_id=None):
-        # type: (str, str, str, [str], str) -> bool
         """
         Updates DDL of a database in Cloud Spanner.
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -286,13 +298,13 @@ def update_database(self, project_id, instance_id, database_id, ddl_statements,
             self.log.error('An error occurred: %s. Exiting.', e.message)
             raise e
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def delete_database(self, project_id, instance_id, database_id):
-        # type: (str, str, str) -> bool
         """
         Drops a database in Cloud Spanner.
 
-        :param project_id:  The ID of the GCP project that owns the Cloud Spanner
-            database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
@@ -319,13 +331,13 @@ def delete_database(self, project_id, instance_id, database_id):
             self.log.info(result)
         return True
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def execute_dml(self, project_id, instance_id, database_id, queries):
-        # type: (str, str, str, str) -> None
         """
         Executes an arbitrary DML query (INSERT, UPDATE, DELETE).
 
-        :param project_id: The ID of the GCP project that owns the Cloud Spanner
-            database.
+        :param project_id: The ID of the  GCP project that owns the Cloud Spanner
+            database. if set to None, default project_id is used from service account.
         :type project_id: str
         :param instance_id: The ID of the Cloud Spanner instance.
         :type instance_id: str
diff --git a/airflow/contrib/hooks/gcp_sql_hook.py b/airflow/contrib/hooks/gcp_sql_hook.py
index af9ad57003..85a6ca026f 100644
--- a/airflow/contrib/hooks/gcp_sql_hook.py
+++ b/airflow/contrib/hooks/gcp_sql_hook.py
@@ -90,11 +90,13 @@ def get_conn(self):
                                http=http_authorized, cache_discovery=False)
         return self._conn
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_instance(self, project_id, instance):
         """
         Retrieves a resource containing information about a Cloud SQL instance.
 
-        :param project_id: Project ID of the project that contains the instance.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param instance: Database instance ID. This does not include the project ID.
         :type instance: str
@@ -106,12 +108,13 @@ def get_instance(self, project_id, instance):
             instance=instance
         ).execute(num_retries=NUM_RETRIES)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def create_instance(self, project_id, body):
         """
         Creates a new Cloud SQL instance.
 
-        :param project_id: Project ID of the project to which the newly created
-            Cloud SQL instances should belong.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param body: Body required by the Cloud SQL insert API, as described in
             https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/insert#request-body.
@@ -126,6 +129,7 @@ def create_instance(self, project_id, body):
         operation_name = response["name"]
         return self._wait_for_operation_to_complete(project_id, operation_name)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def patch_instance(self, project_id, body, instance):
         """
         Updates settings of a Cloud SQL instance.
@@ -133,7 +137,8 @@ def patch_instance(self, project_id, body, instance):
         Caution: This is not a partial update, so you must include values for
         all the settings that you want to retain.
 
-        :param project_id: Project ID of the project that contains the instance.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param body: Body required by the Cloud SQL patch API, as described in
             https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/patch#request-body.
@@ -151,11 +156,13 @@ def patch_instance(self, project_id, body, instance):
         operation_name = response["name"]
         return self._wait_for_operation_to_complete(project_id, operation_name)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def delete_instance(self, project_id, instance):
         """
         Deletes a Cloud SQL instance.
 
-        :param project_id: Project ID of the project that contains the instance.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param instance: Cloud SQL instance ID. This does not include the project ID.
         :type instance: str
@@ -169,11 +176,13 @@ def delete_instance(self, project_id, instance):
         operation_name = response["name"]
         return self._wait_for_operation_to_complete(project_id, operation_name)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def get_database(self, project_id, instance, database):
         """
         Retrieves a database resource from a Cloud SQL instance.
 
-        :param project_id: Project ID of the project that contains the instance.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param instance: Database instance ID. This does not include the project ID.
         :type instance: str
@@ -189,12 +198,14 @@ def get_database(self, project_id, instance, database):
             database=database
         ).execute(num_retries=NUM_RETRIES)
 
-    def create_database(self, project, instance, body):
+    @GoogleCloudBaseHook.fallback_to_default_project_id
+    def create_database(self, project_id, instance, body):
         """
         Creates a new database inside a Cloud SQL instance.
 
-        :param project: Project ID of the project that contains the instance.
-        :type project: str
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
+        :type project_id: str
         :param instance: Database instance ID. This does not include the project ID.
         :type instance: str
         :param body: The request body, as described in
@@ -204,22 +215,24 @@ def create_database(self, project, instance, body):
         :rtype: bool
         """
         response = self.get_conn().databases().insert(
-            project=project,
+            project=project_id,
             instance=instance,
             body=body
         ).execute(num_retries=NUM_RETRIES)
         operation_name = response["name"]
-        return self._wait_for_operation_to_complete(project, operation_name)
+        return self._wait_for_operation_to_complete(project_id, operation_name)
 
-    def patch_database(self, project, instance, database, body):
+    @GoogleCloudBaseHook.fallback_to_default_project_id
+    def patch_database(self, project_id, instance, database, body):
         """
         Updates a database resource inside a Cloud SQL instance.
 
         This method supports patch semantics.
         See https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch.
 
-        :param project: Project ID of the project that contains the instance.
-        :type project: str
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
+        :type project_id: str
         :param instance: Database instance ID. This does not include the project ID.
         :type instance: str
         :param database: Name of the database to be updated in the instance.
@@ -231,20 +244,22 @@ def patch_database(self, project, instance, database, body):
         :rtype: bool
         """
         response = self.get_conn().databases().patch(
-            project=project,
+            project=project_id,
             instance=instance,
             database=database,
             body=body
         ).execute(num_retries=NUM_RETRIES)
         operation_name = response["name"]
-        return self._wait_for_operation_to_complete(project, operation_name)
+        return self._wait_for_operation_to_complete(project_id, operation_name)
 
-    def delete_database(self, project, instance, database):
+    @GoogleCloudBaseHook.fallback_to_default_project_id
+    def delete_database(self, project_id, instance, database):
         """
         Deletes a database from a Cloud SQL instance.
 
-        :param project: Project ID of the project that contains the instance.
-        :type project: str
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
+        :type project_id: str
         :param instance: Database instance ID. This does not include the project ID.
         :type instance: str
         :param database: Name of the database to be deleted in the instance.
@@ -253,19 +268,21 @@ def delete_database(self, project, instance, database):
         :rtype: bool
         """
         response = self.get_conn().databases().delete(
-            project=project,
+            project=project_id,
             instance=instance,
             database=database
         ).execute(num_retries=NUM_RETRIES)
         operation_name = response["name"]
-        return self._wait_for_operation_to_complete(project, operation_name)
+        return self._wait_for_operation_to_complete(project_id, operation_name)
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def export_instance(self, project_id, instance_id, body):
         """
         Exports data from a Cloud SQL instance to a Cloud Storage bucket as a SQL dump
         or CSV file.
 
-        :param project_id: Project ID of the project where the instance exists.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param instance_id: Name of the Cloud SQL instance. This does not include the
             project ID.
@@ -289,12 +306,14 @@ def export_instance(self, project_id, instance_id, body):
                 'Exporting instance {} failed: {}'.format(instance_id, ex.content)
             )
 
+    @GoogleCloudBaseHook.fallback_to_default_project_id
     def import_instance(self, project_id, instance_id, body):
         """
         Imports data into a Cloud SQL instance from a SQL dump or CSV file in
         Cloud Storage.
 
-        :param project_id: Project ID of the project where the instance exists.
+        :param project_id: Project ID of the project that contains the instance. If set
+            to None, default project_id from service account is used.
         :type project_id: str
         :param instance_id: Name of the Cloud SQL instance. This does not include the
             project ID.
@@ -392,7 +411,7 @@ def __init__(self,
             authentication
         :type: str
         :param project_id: Optional id of the GCP project to connect to - it overwrites
-            default project id taken from the GCP connection
+            default project id taken from the GCP connection.
         :type project_id: str
         :param sql_proxy_version: Specific version of SQL proxy to download
             (for example 'v1.13'). By default latest version is downloaded.
@@ -665,7 +684,8 @@ class CloudSqlDatabaseHook(BaseHook):
 
     Remaining parameters are retrieved from the extras (URI query parameters):
 
-    * **project_id** - Google Cloud Platform project where the Cloud SQL instance exists.
+    * **project_id** - Optional, Google Cloud Platform project where the Cloud SQL
+       instance exists. If missing, default project id passed is used.
     * **instance** -  Name of the instance of the Cloud SQL database instance.
     * **location** - The location of the Cloud SQL instance (for example europe-west1).
     * **database_type** - The type of the database instance (MySQL or Postgres).
@@ -682,15 +702,22 @@ class CloudSqlDatabaseHook(BaseHook):
     * **sslcert** - Path to client certificate to authenticate when SSL is used.
     * **sslkey** - Path to client private key to authenticate when SSL is used.
     * **sslrootcert** - Path to server's certificate to authenticate when SSL is used.
+
+    :param gcp_cloudsql_conn_id: URL of the connection
+    :type gcp_cloudsql_conn_id: str
+    :param default_gcp_project_id: Default project id used if project_id not specified
+           in the connection URL
+    :type default_gcp_project_id: str
     """
     _conn = None
 
-    def __init__(self, gcp_cloudsql_conn_id='google_cloud_sql_default'):
+    def __init__(self, gcp_cloudsql_conn_id='google_cloud_sql_default',
+                 default_gcp_project_id=None):
         super(CloudSqlDatabaseHook, self).__init__(source=None)
         self.gcp_cloudsql_conn_id = gcp_cloudsql_conn_id
         self.cloudsql_connection = self.get_connection(self.gcp_cloudsql_conn_id)
         self.extras = self.cloudsql_connection.extra_dejson
-        self.project_id = self.extras.get('project_id')
+        self.project_id = self.extras.get('project_id', default_gcp_project_id)
         self.instance = self.extras.get('instance')
         self.database = self.cloudsql_connection.schema
         self.location = self.extras.get('location')
@@ -737,9 +764,9 @@ def _validate_inputs(self):
         if not self.project_id:
             raise AirflowException("The required extra 'project_id' is empty")
         if not self.location:
-            raise AirflowException("The required extra 'location' is empty")
+            raise AirflowException("The required extra 'location' is empty or None")
         if not self.instance:
-            raise AirflowException("The required extra 'instance' is empty")
+            raise AirflowException("The required extra 'instance' is empty or None")
         if self.database_type not in CLOUD_SQL_VALID_DATABASE_TYPES:
             raise AirflowException("Invalid database type '{}'. Must be one of {}".format(
                 self.database_type, CLOUD_SQL_VALID_DATABASE_TYPES
@@ -754,6 +781,8 @@ def validate_ssl_certs(self):
             self._check_ssl_file(self.sslcert, "sslcert")
             self._check_ssl_file(self.sslkey, "sslkey")
             self._check_ssl_file(self.sslrootcert, "sslrootcert")
+
+    def validate_socket_path_length(self):
         if self.use_proxy and not self.sql_proxy_use_tcp:
             if self.database_type == 'postgres':
                 suffix = "/.s.PGSQL.5432"
diff --git a/airflow/contrib/operators/gcp_bigtable_operator.py b/airflow/contrib/operators/gcp_bigtable_operator.py
index 640851e76e..95c38d3126 100644
--- a/airflow/contrib/operators/gcp_bigtable_operator.py
+++ b/airflow/contrib/operators/gcp_bigtable_operator.py
@@ -44,14 +44,13 @@ def _validate_inputs(self):
 class BigtableInstanceCreateOperator(BaseOperator, BigtableValidationMixin):
     """
     Creates a new Cloud Bigtable instance.
-    If the Cloud Bigtable instance with the given ID exists, the operator does not compare its configuration
+    If the Cloud Bigtable instance with the given ID exists, the operator does not
+    compare its configuration
     and immediately succeeds. No changes are made to the existing instance.
 
     For more details about instance creation have a look at the reference:
     https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.create
 
-    :type project_id: str
-    :param project_id: The ID of the GCP project.
     :type instance_id: str
     :param instance_id: The ID of the Cloud Bigtable instance to create.
     :type main_cluster_id: str
@@ -59,6 +58,8 @@ class BigtableInstanceCreateOperator(BaseOperator, BigtableValidationMixin):
     :type main_cluster_zone: str
     :param main_cluster_zone: The zone for main cluster
         See https://cloud.google.com/bigtable/docs/locations for more details.
+    :type project_id: str
+    :param project_id: Optional, the ID of the GCP project.
     :type replica_cluster_id: str
     :param replica_cluster_id: (optional) The ID for replica cluster for the new instance.
     :type replica_cluster_zone: str
@@ -66,9 +67,11 @@ class BigtableInstanceCreateOperator(BaseOperator, BigtableValidationMixin):
     :type instance_type: IntEnum
     :param instance_type: (optional) The type of the instance.
     :type instance_display_name: str
-    :param instance_display_name: (optional) Human-readable name of the instance. Defaults to ``instance_id``.
+    :param instance_display_name: (optional) Human-readable name of the instance. Defaults
+        to ``instance_id``.
     :type instance_labels: dict
-    :param instance_labels: (optional) Dictionary of labels to associate with the instance.
+    :param instance_labels: (optional) Dictionary of labels to associate
+        with the instance.
     :type cluster_nodes: int
     :param cluster_nodes: (optional) Number of nodes for cluster.
     :type cluster_storage_type: IntEnum
@@ -78,15 +81,17 @@ class BigtableInstanceCreateOperator(BaseOperator, BigtableValidationMixin):
                     If None is not specified, Operator will wait indefinitely.
     """
 
-    REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'main_cluster_id', 'main_cluster_zone')
-    template_fields = ['project_id', 'instance_id', 'main_cluster_id', 'main_cluster_zone']
+    REQUIRED_ATTRIBUTES = ('instance_id', 'main_cluster_id',
+                           'main_cluster_zone')
+    template_fields = ['project_id', 'instance_id', 'main_cluster_id',
+                       'main_cluster_zone']
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  main_cluster_id,
                  main_cluster_zone,
+                 project_id=None,
                  replica_cluster_id=None,
                  replica_cluster_zone=None,
                  instance_display_name=None,
@@ -115,9 +120,11 @@ def __init__(self,
     def execute(self, context):
         instance = self.hook.get_instance(self.project_id, self.instance_id)
         if instance:
-            # Based on Instance.__eq__ instance with the same ID and client is considered as equal.
+            # Based on Instance.__eq__ instance with the same ID and client is
+            # considered as equal.
             self.log.info(
-                "The instance '%s' already exists in this project. Consider it as created",
+                "The instance '%s' already exists in this project. "
+                "Consider it as created",
                 self.instance_id
             )
             return
@@ -148,18 +155,18 @@ class BigtableInstanceDeleteOperator(BaseOperator, BigtableValidationMixin):
     For more details about deleting instance have a look at the reference:
     https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.delete
 
-    :type project_id: str
-    :param project_id: The ID of the GCP project.
     :type instance_id: str
     :param instance_id: The ID of the Cloud Bigtable instance to delete.
+    :param project_id: Optional, the ID of the GCP project.
+    :type project_id: str
     """
-    REQUIRED_ATTRIBUTES = ('project_id', 'instance_id')
+    REQUIRED_ATTRIBUTES = ('instance_id',)
     template_fields = ['project_id', 'instance_id']
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
+                 project_id=None,
                  *args, **kwargs):
         self.project_id = project_id
         self.instance_id = instance_id
@@ -172,7 +179,8 @@ def execute(self, context):
             self.hook.delete_instance(self.project_id, self.instance_id)
         except google.api_core.exceptions.NotFound:
             self.log.info(
-                "The instance '%s' does not exist in project '%s'. Consider it as deleted",
+                "The instance '%s' does not exist in project '%s'. "
+                "Consider it as deleted",
                 self.instance_id, self.project_id
             )
         except google.api_core.exceptions.GoogleAPICallError as e:
@@ -187,27 +195,29 @@ class BigtableTableCreateOperator(BaseOperator, BigtableValidationMixin):
     For more details about creating table have a look at the reference:
     https://googleapis.github.io/google-cloud-python/latest/bigtable/table.html#google.cloud.bigtable.table.Table.create
 
-    :type project_id: str
-    :param project_id: The ID of the GCP project.
     :type instance_id: str
-    :param instance_id: The ID of the Cloud Bigtable instance that will hold the new table.
+    :param instance_id: The ID of the Cloud Bigtable instance that will
+        hold the new table.
     :type table_id: str
     :param table_id: The ID of the table to be created.
+    :type project_id: str
+    :param project_id: Optional, the ID of the GCP project.
     :type initial_split_keys: list
-    :param initial_split_keys: (Optional) list of row keys in bytes that will be used to initially split
-                                the table into several tablets.
+    :param initial_split_keys: (Optional) list of row keys in bytes that will be used to
+        initially split the table into several tablets.
     :type column_families: dict
     :param column_families: (Optional) A map columns to create.
-                            The key is the column_id str and the value is a GarbageCollectionRule
+                            The key is the column_id str and the value is a
+                            GarbageCollectionRule
     """
-    REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'table_id')
+    REQUIRED_ATTRIBUTES = ('instance_id', 'table_id')
     template_fields = ['project_id', 'instance_id', 'table_id']
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  table_id,
+                 project_id=None,
                  initial_split_keys=None,
                  column_families=None,
                  *args, **kwargs):
@@ -222,29 +232,35 @@ def __init__(self,
         super(BigtableTableCreateOperator, self).__init__(*args, **kwargs)
 
     def _compare_column_families(self):
-        table_column_families = self.hook.get_column_families_for_table(self.instance, self.table_id)
+        table_column_families = self.hook.get_column_families_for_table(self.instance,
+                                                                        self.table_id)
         if set(table_column_families.keys()) != set(self.column_families.keys()):
-            self.log.error("Table '%s' has different set of Column Families", self.table_id)
+            self.log.error("Table '%s' has different set of Column Families",
+                           self.table_id)
             self.log.error("Expected: %s", self.column_families.keys())
             self.log.error("Actual: %s", table_column_families.keys())
             return False
 
         for key in table_column_families.keys():
-            # There is difference in structure between local Column Families and remote ones
-            # Local `self.column_families` is dict with column_id as key and GarbageCollectionRule as value.
+            # There is difference in structure between local Column Families
+            # and remote ones
+            # Local `self.column_families` is dict with column_id as key
+            # and GarbageCollectionRule as value.
             # Remote `table_column_families` is list of ColumnFamily objects.
             # For more information about ColumnFamily please refer to the documentation:
             # https://googleapis.github.io/google-cloud-python/latest/bigtable/column-family.html#google.cloud.bigtable.column_family.ColumnFamily
             if table_column_families[key].gc_rule != self.column_families[key]:
-                self.log.error("Column Family '%s' differs for table '%s'.", key, self.table_id)
+                self.log.error("Column Family '%s' differs for table '%s'.", key,
+                               self.table_id)
                 return False
         return True
 
     def execute(self, context):
         self.instance = self.hook.get_instance(self.project_id, self.instance_id)
         if not self.instance:
-            raise AirflowException("Dependency: instance '{}' does not exist in project '{}'.".format(
-                self.instance_id, self.project_id))
+            raise AirflowException(
+                "Dependency: instance '{}' does not exist in project '{}'.".
+                format(self.instance_id, self.project_id))
         try:
             self.hook.create_table(
                 self.instance,
@@ -255,8 +271,10 @@ def execute(self, context):
         except google.api_core.exceptions.AlreadyExists:
             if not self._compare_column_families():
                 raise AirflowException(
-                    "Table '{}' already exists with different Column Families.".format(self.table_id))
-            self.log.info("The table '%s' already exists. Consider it as created", self.table_id)
+                    "Table '{}' already exists with different Column Families.".
+                    format(self.table_id))
+            self.log.info("The table '%s' already exists. Consider it as created",
+                          self.table_id)
 
 
 class BigtableTableDeleteOperator(BaseOperator, BigtableValidationMixin):
@@ -266,21 +284,23 @@ class BigtableTableDeleteOperator(BaseOperator, BigtableValidationMixin):
     For more details about deleting table have a look at the reference:
     https://googleapis.github.io/google-cloud-python/latest/bigtable/table.html#google.cloud.bigtable.table.Table.delete
 
-    :type project_id: str
-    :param project_id: The ID of the GCP project.
     :type instance_id: str
     :param instance_id: The ID of the Cloud Bigtable instance.
     :type table_id: str
     :param table_id: The ID of the table to be deleted.
+    :type project_id: str
+    :param project_id: The ID of the GCP project.
+    :type app_profile_id: str
+    :parm app_profile_id: Application profile.
     """
-    REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'table_id')
+    REQUIRED_ATTRIBUTES = ('instance_id', 'table_id')
     template_fields = ['project_id', 'instance_id', 'table_id']
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  table_id,
+                 project_id=None,
                  app_profile_id=None,
                  *args, **kwargs):
         self.project_id = project_id
@@ -294,7 +314,8 @@ def __init__(self,
     def execute(self, context):
         instance = self.hook.get_instance(self.project_id, self.instance_id)
         if not instance:
-            raise AirflowException("Dependency: instance '{}' does not exist.".format(self.instance_id))
+            raise AirflowException("Dependency: instance '{}' does not exist.".format(
+                self.instance_id))
 
         try:
             self.hook.delete_table(
@@ -304,7 +325,8 @@ def execute(self, context):
             )
         except google.api_core.exceptions.NotFound:
             # It's OK if table doesn't exists.
-            self.log.info("The table '%s' no longer exists. Consider it as deleted", self.table_id)
+            self.log.info("The table '%s' no longer exists. Consider it as deleted",
+                          self.table_id)
         except google.api_core.exceptions.GoogleAPICallError as e:
             self.log.error('An error occurred. Exiting.')
             raise e
@@ -314,27 +336,28 @@ class BigtableClusterUpdateOperator(BaseOperator, BigtableValidationMixin):
     """
     Updates a Cloud Bigtable cluster.
 
-    For more details about updating a Cloud Bigtable cluster, have a look at the reference:
+    For more details about updating a Cloud Bigtable cluster,
+    have a look at the reference:
     https://googleapis.github.io/google-cloud-python/latest/bigtable/cluster.html#google.cloud.bigtable.cluster.Cluster.update
 
-    :type project_id: str
-    :param project_id: The ID of the GCP project.
     :type instance_id: str
     :param instance_id: The ID of the Cloud Bigtable instance.
     :type cluster_id: str
     :param cluster_id: The ID of the Cloud Bigtable cluster to update.
     :type nodes: int
     :param nodes: The desired number of nodes for the Cloud Bigtable cluster.
+    :type project_id: str
+    :param project_id: Optional, the ID of the GCP project.
     """
-    REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'cluster_id', 'nodes')
+    REQUIRED_ATTRIBUTES = ('instance_id', 'cluster_id', 'nodes')
     template_fields = ['project_id', 'instance_id', 'cluster_id', 'nodes']
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  cluster_id,
                  nodes,
+                 project_id=None,
                  *args, **kwargs):
         self.project_id = project_id
         self.instance_id = instance_id
@@ -347,7 +370,8 @@ def __init__(self,
     def execute(self, context):
         instance = self.hook.get_instance(self.project_id, self.instance_id)
         if not instance:
-            raise AirflowException("Dependency: instance '{}' does not exist.".format(self.instance_id))
+            raise AirflowException("Dependency: instance '{}' does not exist.".format(
+                self.instance_id))
 
         try:
             self.hook.update_cluster(
@@ -356,10 +380,9 @@ def execute(self, context):
                 self.nodes
             )
         except google.api_core.exceptions.NotFound:
-            raise AirflowException("Dependency: cluster '{}' does not exist for instance '{}'.".format(
-                self.cluster_id,
-                self.instance_id
-            ))
+            raise AirflowException(
+                "Dependency: cluster '{}' does not exist for instance '{}'.".
+                format(self.cluster_id, self.instance_id))
         except google.api_core.exceptions.GoogleAPICallError as e:
             self.log.error('An error occurred. Exiting.')
             raise e
@@ -373,21 +396,21 @@ class BigtableTableWaitForReplicationSensor(BaseSensorOperator, BigtableValidati
     For more details about cluster states for a table, have a look at the reference:
     https://googleapis.github.io/google-cloud-python/latest/bigtable/table.html#google.cloud.bigtable.table.Table.get_cluster_states
 
-    :type project_id: str
-    :param project_id: The ID of the GCP project.
     :type instance_id: str
     :param instance_id: The ID of the Cloud Bigtable instance.
     :type table_id: str
     :param table_id: The ID of the table to check replication status.
+    :type project_id: str
+    :param project_id: Optional, the ID of the GCP project.
     """
-    REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'table_id')
+    REQUIRED_ATTRIBUTES = ('instance_id', 'table_id')
     template_fields = ['project_id', 'instance_id', 'table_id']
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  table_id,
+                 project_id=None,
                  *args, **kwargs):
         self.project_id = project_id
         self.instance_id = instance_id
@@ -403,10 +426,12 @@ def poke(self, context):
             return False
 
         try:
-            cluster_states = self.hook.get_cluster_states_for_table(instance, self.table_id)
+            cluster_states = self.hook.get_cluster_states_for_table(instance,
+                                                                    self.table_id)
         except google.api_core.exceptions.NotFound:
             self.log.info(
-                "Dependency: table '%s' does not exist in instance '%s'.", self.table_id, self.instance_id)
+                "Dependency: table '%s' does not exist in instance '%s'.",
+                self.table_id, self.instance_id)
             return False
 
         ready_state = ClusterState(enums.Table.ClusterState.ReplicationState.READY)
@@ -414,7 +439,8 @@ def poke(self, context):
         is_table_replicated = True
         for cluster_id in cluster_states.keys():
             if cluster_states[cluster_id] != ready_state:
-                self.log.info("Table '%s' is not yet replicated on cluster '%s'.", self.table_id, cluster_id)
+                self.log.info("Table '%s' is not yet replicated on cluster '%s'.",
+                              self.table_id, cluster_id)
                 is_table_replicated = False
 
         if not is_table_replicated:
diff --git a/airflow/contrib/operators/gcp_compute_operator.py b/airflow/contrib/operators/gcp_compute_operator.py
index a872c17227..b53c2017d1 100644
--- a/airflow/contrib/operators/gcp_compute_operator.py
+++ b/airflow/contrib/operators/gcp_compute_operator.py
@@ -36,16 +36,14 @@ class GceBaseOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  zone,
                  resource_id,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1',
                  *args, **kwargs):
         self.project_id = project_id
         self.zone = zone
-        self.full_location = 'projects/{}/zones/{}'.format(self.project_id,
-                                                           self.zone)
         self.resource_id = resource_id
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
@@ -54,7 +52,7 @@ def __init__(self,
         super(GceBaseOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is missing")
         if not self.zone:
             raise AirflowException("The required parameter 'zone' is missing")
@@ -69,17 +67,21 @@ class GceInstanceStartOperator(GceBaseOperator):
     """
     Starts an instance in Google Compute Engine.
 
-    :param project_id: Google Cloud Platform Project ID where the Compute Engine
-        Instance exists.
-    :type project_id: str
     :param zone: Google Cloud Platform zone where the instance exists.
     :type zone: str
     :param resource_id: Name of the Compute Engine instance resource.
     :type resource_id: str
-    :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
+    :param project_id: Optional, Google Cloud Platform Project ID where the Compute
+        Engine Instance exists.
+    :type project_id: str
+    :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
+        Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
-    :param api_version: API version used (for example v1 or beta).
+    :param api_version: Optional, API version used (for example v1 - or beta). Defaults
+        to v1.
     :type api_version: str
+    :param validate_body: Optional, If set to False, body validation is not performed.
+        Defaults to False.
     """
     # [START gce_instance_start_template_fields]
     template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version')
@@ -87,9 +89,9 @@ class GceInstanceStartOperator(GceBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  zone,
                  resource_id,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1',
                  *args, **kwargs):
@@ -105,17 +107,21 @@ class GceInstanceStopOperator(GceBaseOperator):
     """
     Stops an instance in Google Compute Engine.
 
-    :param project_id: Google Cloud Platform Project ID where the Compute Engine
-        Instance exists.
-    :type project_id: str
     :param zone: Google Cloud Platform zone where the instance exists.
     :type zone: str
     :param resource_id: Name of the Compute Engine instance resource.
     :type resource_id: str
-    :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
+    :param project_id: Optional, Google Cloud Platform Project ID where the Compute
+        Engine Instance exists.
+    :type project_id: str
+    :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
+        Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
-    :param api_version: API version used (for example v1 or beta).
+    :param api_version: Optional, API version used (for example v1 - or beta). Defaults
+        to v1.
     :type api_version: str
+    :param validate_body: Optional, If set to False, body validation is not performed.
+        Defaults to False.
     """
     # [START gce_instance_stop_template_fields]
     template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version')
@@ -123,9 +129,9 @@ class GceInstanceStopOperator(GceBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  zone,
                  resource_id,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1',
                  *args, **kwargs):
@@ -147,9 +153,6 @@ class GceSetMachineTypeOperator(GceBaseOperator):
     Changes the machine type for a stopped instance to the machine type specified in
         the request.
 
-    :param project_id: Google Cloud Platform Project ID where the Compute Engine
-        Instance exists.
-    :type project_id: str
     :param zone: Google Cloud Platform zone where the instance exists.
     :type zone: str
     :param resource_id: Name of the Compute Engine instance resource.
@@ -157,11 +160,17 @@ class GceSetMachineTypeOperator(GceBaseOperator):
     :param body: Body required by the Compute Engine setMachineType API, as described in
         https://cloud.google.com/compute/docs/reference/rest/v1/instances/setMachineType#request-body
     :type body: dict
-    :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
+    :param project_id: Optional, Google Cloud Platform Project ID where the Compute
+        Engine Instance exists.
+    :type project_id: str
+    :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
+        Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
-    :param api_version: API version used (for example v1 or beta).
+    :param api_version: Optional, API version used (for example v1 - or beta). Defaults
+        to v1.
     :type api_version: str
-    :param validate_body: If set to False, body validation is not performed.
+    :param validate_body: Optional, If set to False, body validation is not performed.
+        Defaults to False.
     :type validate_body: bool
     """
     # [START gce_instance_set_machine_type_template_fields]
@@ -170,10 +179,10 @@ class GceSetMachineTypeOperator(GceBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  zone,
                  resource_id,
                  body,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1',
                  validate_body=True,
@@ -249,9 +258,6 @@ class GceInstanceTemplateCopyOperator(GceBaseOperator):
     """
     Copies the instance template, applying specified changes.
 
-    :param project_id: Google Cloud Platform Project ID where the Compute Engine
-        instance exists.
-    :type project_id: str
     :param resource_id: Name of the Instance Template
     :type resource_id: str
     :param body_patch: Patch to the body of instanceTemplates object following rfc7386
@@ -262,16 +268,22 @@ class GceInstanceTemplateCopyOperator(GceBaseOperator):
         - arrays are replaced fully, so if you need to update an array you should
         provide the whole target array as patch element.
     :type body_patch: dict
+    :param project_id: Optional, Google Cloud Platform Project ID where the Compute
+        Engine Instance exists.
+    :type project_id: str
     :param request_id: Optional, unique request_id that you might add to achieve
         full idempotence (for example when client call times out repeating the request
         with the same request id will not create a new instance template again).
         It should be in UUID format as defined in RFC 4122.
     :type request_id: str
-    :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
+    :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
+        Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
-    :param api_version: API version used (for example v1 or beta).
+    :param api_version: Optional, API version used (for example v1 - or beta). Defaults
+        to v1.
     :type api_version: str
-    :param validate_body: If set to False, body validation is not performed.
+    :param validate_body: Optional, If set to False, body validation is not performed.
+        Defaults to False.
     :type validate_body: bool
     """
     # [START gce_instance_template_copy_operator_template_fields]
@@ -281,9 +293,9 @@ class GceInstanceTemplateCopyOperator(GceBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  resource_id,
                  body_patch,
+                 project_id=None,
                  request_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1',
@@ -349,26 +361,31 @@ class GceInstanceGroupManagerUpdateTemplateOperator(GceBaseOperator):
     destination one. API V1 does not have update/patch operations for Instance
     Group Manager, so you must use beta or newer API version. Beta is the default.
 
-    :param project_id: Google Cloud Platform Project ID where the Compute Engine
-        Instance exists.
-    :type project_id: str
     :param resource_id: Name of the Instance Group Manager
     :type resource_id: str
     :param zone: Google Cloud Platform zone where the Instance Group Manager exists.
     :type zone: str
+    :param source_template: URL of the template to replace.
+    :type source_template: str
+    :param destination_template: URL of the target template.
+    :type destination_template: str
+    :param project_id: Optional, Google Cloud Platform Project ID where the Compute
+        Engine Instance exists.
+    :type project_id: str
     :param request_id: Optional, unique request_id that you might add to achieve
         full idempotence (for example when client call times out repeating the request
         with the same request id will not create a new instance template again).
-        It should be in UUID format as defined in RFC 4122
+        It should be in UUID format as defined in RFC 4122.
     :type request_id: str
-    :param update_policy: The update policy for this managed instance group. See
-        https://cloud.google.com/compute/docs/reference/rest/beta/instanceGroupManagers/patch
-        for details of the updatePolicy fields. It's an optional field.
-    :type dict
-    :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
+    :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
+        Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
-    :param api_version: API version used (for example beta).
+    :param api_version: Optional, API version used (for example v1 - or beta). Defaults
+        to v1.
     :type api_version: str
+    :param validate_body: Optional, If set to False, body validation is not performed.
+        Defaults to False.
+    :type validate_body: bool
     """
     # [START gce_igm_update_template_operator_template_fields]
     template_fields = ('project_id', 'resource_id', 'zone', 'request_id',
@@ -378,11 +395,11 @@ class GceInstanceGroupManagerUpdateTemplateOperator(GceBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  resource_id,
                  zone,
                  source_template,
                  destination_template,
+                 project_id=None,
                  update_policy=None,
                  request_id=None,
                  gcp_conn_id='google_cloud_default',
diff --git a/airflow/contrib/operators/gcp_function_operator.py b/airflow/contrib/operators/gcp_function_operator.py
index 7f7da1d3ec..64162a2fc8 100644
--- a/airflow/contrib/operators/gcp_function_operator.py
+++ b/airflow/contrib/operators/gcp_function_operator.py
@@ -81,10 +81,8 @@ def _validate_max_instances(value):
 class GcfFunctionDeployOperator(BaseOperator):
     """
     Creates a function in Google Cloud Functions.
+    If a function with this name already exists, it will be updated.
 
-    :param project_id: Google Cloud Platform Project ID where the function should
-        be created.
-    :type project_id: str
     :param location: Google Cloud Platform region where the function should be created.
     :type location: str
     :param body: Body of the Cloud Functions definition. The body must be a
@@ -93,9 +91,14 @@ class GcfFunctionDeployOperator(BaseOperator):
         . Different API versions require different variants of the Cloud Functions
         dictionary.
     :type body: dict or google.cloud.functions.v1.CloudFunction
-    :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
+    :param project_id: (Optional) Google Cloud Platform project ID where the function
+        should be created.
+    :type project_id: str
+    :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud
+         Platform - default 'google_cloud_default'.
     :type gcp_conn_id: str
-    :param api_version: API version used (for example v1 or v1beta1).
+    :param api_version: (Optional) API version used (for example v1 - default -  or
+        v1beta1).
     :type api_version: str
     :param zip_path: Path to zip file containing source code of the function. If the path
         is set, the sourceUploadUrl should not be specified in the body or it should
@@ -105,15 +108,15 @@ class GcfFunctionDeployOperator(BaseOperator):
     :param validate_body: If set to False, body validation is not performed.
     :type validate_body: bool
     """
-    # [START gce_function_deploy_template_operator_template_fields]
+    # [START gcf_function_deploy_template_fields]
     template_fields = ('project_id', 'location', 'gcp_conn_id', 'api_version')
-    # [END gce_function_deploy_template_operator_template_fields]
+    # [END gcf_function_deploy_template_fields]
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  location,
                  body,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1',
                  zip_path=None,
@@ -121,8 +124,6 @@ def __init__(self,
                  *args, **kwargs):
         self.project_id = project_id
         self.location = location
-        self.full_location = 'projects/{}/locations/{}'.format(self.project_id,
-                                                               self.location)
         self.body = body
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
@@ -137,8 +138,6 @@ def __init__(self,
         super(GcfFunctionDeployOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
-            raise AirflowException("The required parameter 'project_id' is missing")
         if not self.location:
             raise AirflowException("The required parameter 'location' is missing")
         if not self.body:
@@ -150,7 +149,7 @@ def _validate_all_body_fields(self):
             self._field_validator.validate(self.body)
 
     def _create_new_function(self):
-        self._hook.create_new_function(self.full_location, self.body)
+        self._hook.create_new_function(self.project_id, self.location, self.body)
 
     def _update_function(self):
         self._hook.update_function(self.body['name'], self.body, self.body.keys())
@@ -170,7 +169,8 @@ def _check_if_function_exists(self):
         return True
 
     def _upload_source_code(self):
-        return self._hook.upload_function_zip(parent=self.full_location,
+        return self._hook.upload_function_zip(project_id=self.project_id,
+                                              location=self.location,
                                               zip_path=self.zip_path)
 
     def _set_airflow_version_label(self):
@@ -181,7 +181,7 @@ def _set_airflow_version_label(self):
 
     def execute(self, context):
         if self.zip_path_preprocessor.should_upload_function():
-            self.body[SOURCE_UPLOAD_URL] = self._upload_source_code()
+            self.body[GCF_SOURCE_UPLOAD_URL] = self._upload_source_code()
         self._validate_all_body_fields()
         self._set_airflow_version_label()
         if not self._check_if_function_exists():
@@ -190,10 +190,10 @@ def execute(self, context):
             self._update_function()
 
 
-SOURCE_ARCHIVE_URL = 'sourceArchiveUrl'
-SOURCE_UPLOAD_URL = 'sourceUploadUrl'
+GCF_SOURCE_ARCHIVE_URL = 'sourceArchiveUrl'
+GCF_SOURCE_UPLOAD_URL = 'sourceUploadUrl'
 SOURCE_REPOSITORY = 'sourceRepository'
-ZIP_PATH = 'zip_path'
+GCF_ZIP_PATH = 'zip_path'
 
 
 class ZipPathPreprocessor:
@@ -226,28 +226,28 @@ def _is_present_and_empty(dictionary, field):
         return field in dictionary and not dictionary[field]
 
     def _verify_upload_url_and_no_zip_path(self):
-        if self._is_present_and_empty(self.body, SOURCE_UPLOAD_URL):
+        if self._is_present_and_empty(self.body, GCF_SOURCE_UPLOAD_URL):
             if not self.zip_path:
                 raise AirflowException(
                     "Parameter '{}' is empty in the body and argument '{}' "
                     "is missing or empty. You need to have non empty '{}' "
                     "when '{}' is present and empty.".
-                    format(SOURCE_UPLOAD_URL, ZIP_PATH, ZIP_PATH, SOURCE_UPLOAD_URL))
+                    format(GCF_SOURCE_UPLOAD_URL, GCF_ZIP_PATH, GCF_ZIP_PATH, GCF_SOURCE_UPLOAD_URL))
 
     def _verify_upload_url_and_zip_path(self):
-        if SOURCE_UPLOAD_URL in self.body and self.zip_path:
-            if not self.body[SOURCE_UPLOAD_URL]:
+        if GCF_SOURCE_UPLOAD_URL in self.body and self.zip_path:
+            if not self.body[GCF_SOURCE_UPLOAD_URL]:
                 self.upload_function = True
             else:
                 raise AirflowException("Only one of '{}' in body or '{}' argument "
                                        "allowed. Found both."
-                                       .format(SOURCE_UPLOAD_URL, ZIP_PATH))
+                                       .format(GCF_SOURCE_UPLOAD_URL, GCF_ZIP_PATH))
 
     def _verify_archive_url_and_zip_path(self):
-        if SOURCE_ARCHIVE_URL in self.body and self.zip_path:
+        if GCF_SOURCE_ARCHIVE_URL in self.body and self.zip_path:
             raise AirflowException("Only one of '{}' in body or '{}' argument "
                                    "allowed. Found both."
-                                   .format(SOURCE_ARCHIVE_URL, ZIP_PATH))
+                                   .format(GCF_SOURCE_ARCHIVE_URL, GCF_ZIP_PATH))
 
     def should_upload_function(self):
         if self.upload_function is None:
@@ -279,9 +279,9 @@ class GcfFunctionDeleteOperator(BaseOperator):
     :param api_version: API version used (for example v1 or v1beta1).
     :type api_version: str
     """
-    # [START gce_function_delete_template_operator_template_fields]
+    # [START gcf_function_delete_template_fields]
     template_fields = ('name', 'gcp_conn_id', 'api_version')
-    # [END gce_function_delete_template_operator_template_fields]
+    # [END gcf_function_delete_template_fields]
 
     @apply_defaults
     def __init__(self,
diff --git a/airflow/contrib/operators/gcp_spanner_operator.py b/airflow/contrib/operators/gcp_spanner_operator.py
index 1cb4c7d47f..429adf5063 100644
--- a/airflow/contrib/operators/gcp_spanner_operator.py
+++ b/airflow/contrib/operators/gcp_spanner_operator.py
@@ -29,8 +29,6 @@ class CloudSpannerInstanceDeployOperator(BaseOperator):
     Creates a new Cloud Spanner instance, or if an instance with the same instance_id
     exists in the specified project, updates the Cloud Spanner instance.
 
-    :param project_id: The ID of the project which owns the Cloud Spanner Database.
-    :type project_id: str
     :param instance_id: Cloud Spanner instance ID.
     :type instance_id: str
     :param configuration_name:  The name of the Cloud Spanner instance configuration
@@ -44,6 +42,9 @@ class CloudSpannerInstanceDeployOperator(BaseOperator):
       the GCP Console. (Must be between 4 and 30 characters.) If this value is not set
       in the constructor, the name is the same as the instance ID.
     :type display_name: str
+    :param project_id: Optional, the ID of the project which owns the Cloud Spanner
+        Database.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     """
@@ -54,11 +55,11 @@ class CloudSpannerInstanceDeployOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  configuration_name,
                  node_count,
                  display_name,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  *args, **kwargs):
         self.instance_id = instance_id
@@ -72,10 +73,11 @@ def __init__(self,
         super(CloudSpannerInstanceDeployOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance_id:
-            raise AirflowException("The required parameter 'instance_id' is empty")
+            raise AirflowException("The required parameter 'instance_id' "
+                                   "is empty or None")
 
     def execute(self, context):
         if not self._hook.get_instance(self.project_id, self.instance_id):
@@ -96,10 +98,11 @@ class CloudSpannerInstanceDeleteOperator(BaseOperator):
     Deletes a Cloud Spanner instance. If an instance does not exist,
     no action is taken and the operator succeeds.
 
-    :param project_id: The ID of the project that owns the Cloud Spanner Database.
-    :type project_id: str
     :param instance_id: The Cloud Spanner instance ID.
     :type instance_id: str
+    :param project_id: Optional, the ID of the project that owns the Cloud Spanner
+        Database.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     """
@@ -109,8 +112,8 @@ class CloudSpannerInstanceDeleteOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  *args, **kwargs):
         self.instance_id = instance_id
@@ -121,10 +124,11 @@ def __init__(self,
         super(CloudSpannerInstanceDeleteOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance_id:
-            raise AirflowException("The required parameter 'instance_id' is empty")
+            raise AirflowException("The required parameter 'instance_id' "
+                                   "is empty or None")
 
     def execute(self, context):
         if self._hook.get_instance(self.project_id, self.instance_id):
@@ -140,8 +144,6 @@ class CloudSpannerInstanceDatabaseQueryOperator(BaseOperator):
     """
     Executes an arbitrary DML query (INSERT, UPDATE, DELETE).
 
-    :param project_id: The ID of the project that owns the Cloud Spanner Database.
-    :type project_id: str
     :param instance_id: The Cloud Spanner instance ID.
     :type instance_id: str
     :param database_id: The Cloud Spanner database ID.
@@ -149,6 +151,9 @@ class CloudSpannerInstanceDatabaseQueryOperator(BaseOperator):
     :param query: The query or list of queries to be executed. Can be a path to a SQL
        file.
     :type query: str or list
+    :param project_id: Optional, the ID of the project that owns the Cloud Spanner
+        Database.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     """
@@ -159,10 +164,10 @@ class CloudSpannerInstanceDatabaseQueryOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  database_id,
                  query,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  *args, **kwargs):
         self.instance_id = instance_id
@@ -175,12 +180,14 @@ def __init__(self,
         super(CloudSpannerInstanceDatabaseQueryOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance_id:
-            raise AirflowException("The required parameter 'instance_id' is empty")
+            raise AirflowException("The required parameter 'instance_id' "
+                                   "is empty or None")
         if not self.database_id:
-            raise AirflowException("The required parameter 'database_id' is empty")
+            raise AirflowException("The required parameter 'database_id' "
+                                   "is empty or None")
         if not self.query:
             raise AirflowException("The required parameter 'query' is empty")
 
@@ -207,14 +214,15 @@ class CloudSpannerInstanceDatabaseDeployOperator(BaseOperator):
     Creates a new Cloud Spanner database, or if database exists,
     the operator does nothing.
 
-    :param project_id: The ID of the project that owns the Cloud Spanner Database.
-    :type project_id: str
     :param instance_id: The Cloud Spanner instance ID.
     :type instance_id: str
     :param database_id: The Cloud Spanner database ID.
     :type database_id: str
     :param ddl_statements: The string list containing DDL for the new database.
     :type ddl_statements: list of str
+    :param project_id: Optional, the ID of the project that owns the Cloud Spanner
+        Database.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     """
@@ -226,10 +234,10 @@ class CloudSpannerInstanceDatabaseDeployOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  database_id,
                  ddl_statements,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  *args, **kwargs):
         self.instance_id = instance_id
@@ -242,14 +250,14 @@ def __init__(self,
         super(CloudSpannerInstanceDatabaseDeployOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance_id:
-            raise AirflowException("The required parameter 'instance_id' is empty")
+            raise AirflowException("The required parameter 'instance_id' is empty "
+                                   "or None")
         if not self.database_id:
-            raise AirflowException("The required parameter 'database_id' is empty")
-        if not self.ddl_statements:
-            raise AirflowException("The required parameter 'ddl_statements' is empty")
+            raise AirflowException("The required parameter 'database_id' is empty"
+                                   " or None")
 
     def execute(self, context):
         if not self._hook.get_database(self.project_id,
@@ -273,14 +281,15 @@ class CloudSpannerInstanceDatabaseUpdateOperator(BaseOperator):
     """
     Updates a Cloud Spanner database with the specified DDL statement.
 
-    :param project_id: The ID of the project that owns the the Cloud Spanner Database.
-    :type project_id: str
     :param instance_id: The Cloud Spanner instance ID.
     :type instance_id: str
     :param database_id: The Cloud Spanner database ID.
     :type database_id: str
     :param ddl_statements: The string list containing DDL to apply to the database.
     :type ddl_statements: list[str]
+    :param project_id: Optional, the ID of the project that owns the the Cloud Spanner
+        Database.
+    :type project_id: str
     :param operation_id: (Optional) Unique per database operation id that can
            be specified to implement idempotency check.
     :type operation_id: str
@@ -295,10 +304,10 @@ class CloudSpannerInstanceDatabaseUpdateOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  database_id,
                  ddl_statements,
+                 project_id=None,
                  operation_id=None,
                  gcp_conn_id='google_cloud_default',
                  *args, **kwargs):
@@ -313,14 +322,17 @@ def __init__(self,
         super(CloudSpannerInstanceDatabaseUpdateOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance_id:
-            raise AirflowException("The required parameter 'instance_id' is empty")
+            raise AirflowException("The required parameter 'instance_id' is empty"
+                                   " or None")
         if not self.database_id:
-            raise AirflowException("The required parameter 'database_id' is empty")
+            raise AirflowException("The required parameter 'database_id' is empty"
+                                   " or None")
         if not self.ddl_statements:
-            raise AirflowException("The required parameter 'ddl_statements' is empty")
+            raise AirflowException("The required parameter 'ddl_statements' is empty"
+                                   " or None")
 
     def execute(self, context):
         if not self._hook.get_database(self.project_id,
@@ -343,12 +355,13 @@ class CloudSpannerInstanceDatabaseDeleteOperator(BaseOperator):
     """
     Deletes a Cloud Spanner database.
 
-    :param project_id: The ID of the project that owns the Cloud Spanner Database.
-    :type project_id: str
     :param instance_id: Cloud Spanner instance ID.
     :type instance_id: str
     :param database_id: Cloud Spanner database ID.
     :type database_id: str
+    :param project_id: Optional, the ID of the project that owns the Cloud Spanner
+        Database.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     """
@@ -359,9 +372,9 @@ class CloudSpannerInstanceDatabaseDeleteOperator(BaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance_id,
                  database_id,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  *args, **kwargs):
         self.instance_id = instance_id
@@ -373,12 +386,14 @@ def __init__(self,
         super(CloudSpannerInstanceDatabaseDeleteOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance_id:
-            raise AirflowException("The required parameter 'instance_id' is empty")
+            raise AirflowException("The required parameter 'instance_id' is empty"
+                                   " or None")
         if not self.database_id:
-            raise AirflowException("The required parameter 'database_id' is empty")
+            raise AirflowException("The required parameter 'database_id' is empty"
+                                   " or None")
 
     def execute(self, context):
         db = self._hook.get_database(self.project_id,
diff --git a/airflow/contrib/operators/gcp_sql_operator.py b/airflow/contrib/operators/gcp_sql_operator.py
index e5c6cb8dbc..e183a10d04 100644
--- a/airflow/contrib/operators/gcp_sql_operator.py
+++ b/airflow/contrib/operators/gcp_sql_operator.py
@@ -23,6 +23,7 @@
 from airflow.contrib.utils.gcp_field_validator import GcpBodyFieldValidator
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
+from airflow.hooks.base_hook import BaseHook
 
 SETTINGS = 'settings'
 SETTINGS_VERSION = 'settingsVersion'
@@ -136,10 +137,10 @@ class CloudSqlBaseOperator(BaseOperator):
     """
     Abstract base operator for Google Cloud SQL operators to inherit from.
 
-    :param project_id: Project ID of the Google Cloud Platform project to operate it.
-    :type project_id: str
     :param instance: Cloud SQL instance ID. This does not include the project ID.
     :type instance: str
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -147,8 +148,8 @@ class CloudSqlBaseOperator(BaseOperator):
     """
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  *args, **kwargs):
@@ -162,10 +163,10 @@ def __init__(self,
         super(CloudSqlBaseOperator, self).__init__(*args, **kwargs)
 
     def _validate_inputs(self):
-        if not self.project_id:
+        if self.project_id == '':
             raise AirflowException("The required parameter 'project_id' is empty")
         if not self.instance:
-            raise AirflowException("The required parameter 'instance' is empty")
+            raise AirflowException("The required parameter 'instance' is empty or None")
 
     def _check_if_instance_exists(self, instance):
         try:
@@ -199,15 +200,14 @@ class CloudSqlInstanceCreateOperator(CloudSqlBaseOperator):
     If an instance with the same name exists, no action will be taken and
     the operator will succeed.
 
-    :param project_id: Project ID of the project to which the newly created Cloud SQL
-        instances should belong.
-    :type project_id: str
     :param body: Body required by the Cloud SQL insert API, as described in
         https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/insert
         #request-body
     :type body: dict
     :param instance: Cloud SQL instance ID. This does not include the project ID.
     :type instance: str
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -221,9 +221,9 @@ class CloudSqlInstanceCreateOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  body,
                  instance,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  validate_body=True,
@@ -269,13 +269,13 @@ class CloudSqlInstancePatchOperator(CloudSqlBaseOperator):
     to the rules of patch semantics.
     https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch
 
-    :param project_id: Project ID of the project that contains the instance.
-    :type project_id: str
     :param body: Body required by the Cloud SQL patch API, as described in
         https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/patch#request-body
     :type body: dict
     :param instance: Cloud SQL instance ID. This does not include the project ID.
     :type instance: str
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -287,9 +287,9 @@ class CloudSqlInstancePatchOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  body,
                  instance,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  *args, **kwargs):
@@ -316,10 +316,10 @@ class CloudSqlInstanceDeleteOperator(CloudSqlBaseOperator):
     """
     Deletes a Cloud SQL instance.
 
-    :param project_id: Project ID of the project that contains the instance to be deleted.
-    :type project_id: str
     :param instance: Cloud SQL instance ID. This does not include the project ID.
     :type instance: str
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -331,8 +331,8 @@ class CloudSqlInstanceDeleteOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  *args, **kwargs):
@@ -353,13 +353,13 @@ class CloudSqlInstanceDatabaseCreateOperator(CloudSqlBaseOperator):
     """
     Creates a new database inside a Cloud SQL instance.
 
-    :param project_id: Project ID of the project that contains the instance.
-    :type project_id: str
     :param instance: Database instance ID. This does not include the project ID.
     :type instance: str
     :param body: The request body, as described in
         https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases/insert#request-body
     :type body: dict
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -373,9 +373,9 @@ class CloudSqlInstanceDatabaseCreateOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
                  body,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  validate_body=True,
@@ -419,8 +419,6 @@ class CloudSqlInstanceDatabasePatchOperator(CloudSqlBaseOperator):
     instance using patch semantics.
     See: https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch
 
-    :param project_id: Project ID of the project that contains the instance.
-    :type project_id: str
     :param instance: Database instance ID. This does not include the project ID.
     :type instance: str
     :param database: Name of the database to be updated in the instance.
@@ -428,6 +426,8 @@ class CloudSqlInstanceDatabasePatchOperator(CloudSqlBaseOperator):
     :param body: The request body, as described in
         https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases/patch#request-body
     :type body: dict
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -442,10 +442,10 @@ class CloudSqlInstanceDatabasePatchOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
                  database,
                  body,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  validate_body=True,
@@ -485,12 +485,12 @@ class CloudSqlInstanceDatabaseDeleteOperator(CloudSqlBaseOperator):
     """
     Deletes a database from a Cloud SQL instance.
 
-    :param project_id: Project ID of the project that contains the instance.
-    :type project_id: str
     :param instance: Database instance ID. This does not include the project ID.
     :type instance: str
     :param database: Name of the database to be deleted in the instance.
     :type database: str
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -503,9 +503,9 @@ class CloudSqlInstanceDatabaseDeleteOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
                  database,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  *args, **kwargs):
@@ -538,14 +538,13 @@ class CloudSqlInstanceExportOperator(CloudSqlBaseOperator):
     Note: This operator is idempotent. If executed multiple times with the same
     export file URI, the export file in GCS will simply be overridden.
 
-    :param project_id: Project ID of the project that contains the instance to be
-        exported.
-    :type project_id: str
     :param instance: Cloud SQL instance ID. This does not include the project ID.
     :type instance: str
     :param body: The request body, as described in
         https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/export#request-body
     :type body: dict
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -559,9 +558,9 @@ class CloudSqlInstanceExportOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
                  body,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  validate_body=True,
@@ -607,13 +606,13 @@ class CloudSqlInstanceImportOperator(CloudSqlBaseOperator):
     If the import file was generated in a different way, idempotence is not guaranteed.
     It has to be ensured on the SQL file level.
 
-    :param project_id: Project ID of the project that contains the instance.
-    :type project_id: str
     :param instance: Cloud SQL instance ID. This does not include the project ID.
     :type instance: str
     :param body: The request body, as described in
         https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/export#request-body
     :type body: dict
+    :param project_id: Optional, Google Cloud Platform Project ID.
+    :type project_id: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
@@ -627,9 +626,9 @@ class CloudSqlInstanceImportOperator(CloudSqlBaseOperator):
 
     @apply_defaults
     def __init__(self,
-                 project_id,
                  instance,
                  body,
+                 project_id=None,
                  gcp_conn_id='google_cloud_default',
                  api_version='v1beta4',
                  validate_body=True,
@@ -699,8 +698,11 @@ def __init__(self,
         self.gcp_cloudsql_conn_id = gcp_cloudsql_conn_id
         self.autocommit = autocommit
         self.parameters = parameters
+        self.gcp_connection = BaseHook.get_connection(self.gcp_conn_id)
         self.cloudsql_db_hook = CloudSqlDatabaseHook(
-            gcp_cloudsql_conn_id=gcp_cloudsql_conn_id)
+            gcp_cloudsql_conn_id=gcp_cloudsql_conn_id,
+            default_gcp_project_id=self.gcp_connection.extra_dejson.get(
+                'extra__google_cloud_platform__project'))
         self.cloud_sql_proxy_runner = None
         self.database_hook = None
 
@@ -708,6 +710,7 @@ def execute(self, context):
         self.cloudsql_db_hook.validate_ssl_certs()
         self.cloudsql_db_hook.create_connection()
         try:
+            self.cloudsql_db_hook.validate_socket_path_length()
             self.database_hook = self.cloudsql_db_hook.get_database_hook()
             try:
                 try:
diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst
index bcb9f9b9b2..b2828662dc 100644
--- a/docs/howto/operator.rst
+++ b/docs/howto/operator.rst
@@ -144,12 +144,24 @@ The following examples of OS environment variables used to pass arguments to the
 Using the operator
 """"""""""""""""""
 
+The code to create the operator:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gce_start]
     :end-before: [END howto_operator_gce_start]
 
+You can also create the operator without project id - project id will be retrieved
+from the service account used:
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gce_start_no_project_id]
+    :end-before: [END howto_operator_gce_start_no_project_id]
+
+
 Templating
 """"""""""
 
@@ -187,12 +199,23 @@ The following examples of OS environment variables used to pass arguments to the
 Using the operator
 """"""""""""""""""
 
+The code to create the operator:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gce_stop]
     :end-before: [END howto_operator_gce_stop]
 
+You can also create the operator without project id - project id will be retrieved
+from the service account used:
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gce_stop_no_project_id]
+    :end-before: [END howto_operator_gce_stop_no_project_id]
+
 Templating
 """"""""""
 
@@ -236,12 +259,23 @@ The following examples of OS environment variables used to pass arguments to the
 Using the operator
 """"""""""""""""""
 
+The code to create the operator:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gce_set_machine_type]
     :end-before: [END howto_operator_gce_set_machine_type]
 
+You can also create the operator without project id - project id will be retrieved
+from the service account used:
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gce_set_machine_type_no_project_id]
+    :end-before: [END howto_operator_gce_set_machine_type_no_project_id]
+
 Templating
 """"""""""
 
@@ -285,12 +319,23 @@ The following examples of OS environment variables used to pass arguments to the
 Using the operator
 """"""""""""""""""
 
+The code to create the operator:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gce_igm_copy_template]
     :end-before: [END howto_operator_gce_igm_copy_template]
 
+You can also create the operator without project id - project id will be retrieved
+from the service account used:
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gce_igm_copy_template_no_project_id]
+    :end-before: [END howto_operator_gce_igm_copy_template_no_project_id]
+
 Templating
 """"""""""
 
@@ -332,12 +377,24 @@ The following examples of OS environment variables used to pass arguments to the
 Using the operator
 """"""""""""""""""
 
+The code to create the operator:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gce_igm_update_template]
     :end-before: [END howto_operator_gce_igm_update_template]
 
+You can also create the operator without project id - project id will be retrieved
+from the service account used:
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gce_igm_update_template_no_project_id]
+    :end-before: [END howto_operator_gce_igm_update_template_no_project_id]
+
+
 Templating
 """"""""""
 
@@ -387,13 +444,15 @@ and immediately succeeds. No changes are made to the existing instance.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gcp_bigtable_instance_create]
     :end-before: [END howto_operator_gcp_bigtable_instance_create]
 
-
 BigtableInstanceDeleteOperator
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
@@ -403,6 +462,9 @@ to delete a Google Cloud Bigtable instance.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py
     :language: python
     :dedent: 4
@@ -418,13 +480,15 @@ to modify number of nodes in a Cloud Bigtable cluster.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gcp_bigtable_cluster_update]
     :end-before: [END howto_operator_gcp_bigtable_cluster_update]
 
-
 BigtableTableCreateOperator
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
@@ -438,6 +502,9 @@ error message.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py
     :language: python
     :dedent: 4
@@ -462,6 +529,9 @@ to delete a table in Google Cloud Bigtable.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py
     :language: python
     :dedent: 4
@@ -471,6 +541,9 @@ Using the operator
 BigtableTableWaitForReplicationSensor
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 Use the :class:`~airflow.contrib.operators.gcp_bigtable_operator.BigtableTableWaitForReplicationSensor`
 to wait for the table to replicate fully.
 
@@ -488,8 +561,6 @@ Using the operator
     :start-after: [START howto_operator_gcp_bigtable_table_wait_for_replication]
     :end-before: [END howto_operator_gcp_bigtable_table_wait_for_replication]
 
-
-
 Google Cloud Functions Operators
 --------------------------------
 
@@ -507,15 +578,15 @@ Arguments
 The following examples of OS environment variables show how you can build function name
 to use in the operator:
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
-    :start-after: [START howto_operator_gcf_delete_args]
-    :end-before: [END howto_operator_gcf_delete_args]
+    :start-after: [START howto_operator_gcf_common_variables]
+    :end-before: [END howto_operator_gcf_common_variables]
 
 Using the operator
 """"""""""""""""""
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gcf_delete]
@@ -527,33 +598,8 @@ Templating
 .. literalinclude:: ../../airflow/contrib/operators/gcp_function_operator.py
     :language: python
     :dedent: 4
-    :start-after: [START gce_function_delete_template_operator_template_fields]
-    :end-before: [END gce_function_delete_template_operator_template_fields]
-
-Troubleshooting
-"""""""""""""""
-If you want to run or deploy an operator using a service account and get “forbidden 403”
-errors, it means that your service account does not have the correct
-Cloud IAM permissions.
-
-1. Assign your Service Account the Cloud Functions Developer role.
-2. Grant the user the Cloud IAM Service Account User role on the Cloud Functions runtime
-   service account.
-
-The typical way of assigning Cloud IAM permissions with `gcloud` is
-shown below. Just replace PROJECT_ID with ID of your Google Cloud Platform project
-and SERVICE_ACCOUNT_EMAIL with the email ID of your service account.
-
-.. code-block:: bash
-
-  gcloud iam service-accounts add-iam-policy-binding \
-    PROJECT_ID@appspot.gserviceaccount.com \
-    --member="serviceAccount:[SERVICE_ACCOUNT_EMAIL]" \
-    --role="roles/iam.serviceAccountUser"
-
-
-See `Adding the IAM service agent user role to the runtime service
-<https://cloud.google.com/functions/docs/reference/iam/roles#adding_the_iam_service_agent_user_role_to_the_runtime_service_account>`_.
+    :start-after: [START gcf_function_delete_template_fields]
+    :end-before: [END gcf_function_delete_template_fields]
 
 More information
 """"""""""""""""
@@ -565,6 +611,7 @@ GcfFunctionDeployOperator
 ^^^^^^^^^^^^^^^^^^^^^^^^^
 
 Use the operator to deploy a function to Google Cloud Functions.
+If a function with this name already exists, it will be updated.
 
 For parameter definition, take a look at
 :class:`~airflow.contrib.operators.gcp_function_operator.GcfFunctionDeployOperator`.
@@ -573,76 +620,95 @@ For parameter definition, take a look at
 Arguments
 """""""""
 
-The following examples of OS environment variables show several variants of args you can
-use with the operator:
+In the example DAG the following environment variables are used to parameterize the
+operator's definition:
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
+    :language: python
+    :start-after: [START howto_operator_gcf_common_variables]
+    :end-before: [END howto_operator_gcf_common_variables]
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
     :start-after: [START howto_operator_gcf_deploy_variables]
     :end-before: [END howto_operator_gcf_deploy_variables]
 
-With those variables you can define the body of the request:
+Some of those variables are used to create the request's body:
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
     :start-after: [START howto_operator_gcf_deploy_body]
     :end-before: [END howto_operator_gcf_deploy_body]
 
-When you create a DAG, the default_args dictionary can be used to pass
+When a DAG is created, the default_args dictionary can be used to pass
 arguments common with other tasks:
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
     :start-after: [START howto_operator_gcf_default_args]
     :end-before: [END howto_operator_gcf_default_args]
 
 Note that the neither the body nor the default args are complete in the above examples.
 Depending on the variables set, there might be different variants on how to pass source
-code related fields. Currently, you can pass either sourceArchiveUrl, sourceRepository
-or sourceUploadUrl as described in the
+code related fields. Currently, you can pass either ``sourceArchiveUrl``,
+``sourceRepository`` or ``sourceUploadUrl`` as described in the
 `Cloud Functions API specification
 <https://cloud.google.com/functions/docs/reference/rest/v1/projects.locations.functions#CloudFunction>`_.
 
-Additionally, default_args or direct operator args might contain zip_path parameter
+Additionally, ``default_args`` or direct operator args might contain ``zip_path``
+parameter
 to run the extra step of uploading the source code before deploying it.
-In this case, you also need to provide an empty `sourceUploadUrl`
+In this case, you also need to provide an empty ``sourceUploadUrl``
 parameter in the body.
 
 Using the operator
 """"""""""""""""""
 
-Based on the variables defined above, example logic of setting the source code
-related fields is shown here:
+Depending on the combination of parameters, the Function's source code can be obtained
+from different sources:
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
     :start-after: [START howto_operator_gcf_deploy_variants]
     :end-before: [END howto_operator_gcf_deploy_variants]
 
 The code to create the operator:
 
-.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_gcf_deploy]
     :end-before: [END howto_operator_gcf_deploy]
 
+You can also create the operator without project id - project id will be retrieved
+from the service account used:
+
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gcf_deploy_no_project_id]
+    :end-before: [END howto_operator_gcf_deploy_no_project_id]
+
 Templating
 """"""""""
 
 .. literalinclude:: ../../airflow/contrib/operators/gcp_function_operator.py
     :language: python
     :dedent: 4
-    :start-after: [START gce_function_deploy_template_operator_template_fields]
-    :end-before: [END gce_function_deploy_template_operator_template_fields]
+    :start-after: [START gcf_function_deploy_template_fields]
+    :end-before: [END gcf_function_deploy_template_fields]
 
 
 Troubleshooting
 """""""""""""""
 
-If you want to run or deploy an operator using a service account and get “forbidden 403”
-errors, it means that your service account does not have the correct
-Cloud IAM permissions.
+If during the deploy you see an error similar to:
+
+`"HttpError 403: Missing necessary permission iam.serviceAccounts.actAs for on resource
+project-name@appspot.gserviceaccount.com. Please grant the
+roles/iam.serviceAccountUser role."`
+
+it means that your service account does not have the correct Cloud IAM permissions.
 
 1. Assign your Service Account the Cloud Functions Developer role.
 2. Grant the user the Cloud IAM Service Account User role on the Cloud Functions runtime
@@ -659,7 +725,9 @@ and SERVICE_ACCOUNT_EMAIL with the email ID of your service account.
     --member="serviceAccount:[SERVICE_ACCOUNT_EMAIL]" \
     --role="roles/iam.serviceAccountUser"
 
-See `Adding the IAM service agent user role to the runtime service <https://cloud.google.com/functions/docs/reference/iam/roles#adding_the_iam_service_agent_user_role_to_the_runtime_service_account>`_  for details
+You can also do that via the GCP Web console.
+
+See `Adding the IAM service agent user role to the runtime service <https://cloud.google.com/functions/docs/reference/iam/roles#adding_the_iam_service_agent_user_role_to_the_runtime_service_account>`_  for details.
 
 If the source code for your function is in Google Source Repository, make sure that
 your service account has the Source Repository Viewer role so that the source code
@@ -696,6 +764,9 @@ Some arguments in the example DAG are taken from environment variables.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py
     :language: python
     :dedent: 4
@@ -742,6 +813,9 @@ Some arguments in the example DAG are taken from environment variables.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py
     :language: python
     :dedent: 4
@@ -791,12 +865,21 @@ Some arguments in the example DAG are taken from environment variables.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_spanner_database_update]
     :end-before: [END howto_operator_spanner_database_update]
 
+.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_spanner_database_update_idempotent]
+    :end-before: [END howto_operator_spanner_database_update_idempotent]
+
 Templating
 """"""""""
 
@@ -833,6 +916,9 @@ Some arguments in the example DAG are taken from environment variables.
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py
     :language: python
     :dedent: 4
@@ -876,6 +962,9 @@ Some arguments in the example DAG are taken from environment variables:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py
     :language: python
     :dedent: 4
@@ -921,6 +1010,9 @@ Some arguments in the example DAG are taken from environment variables:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -970,6 +1062,9 @@ Some arguments in the example DAG are taken from environment variables:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -1014,6 +1109,9 @@ Some arguments in the example DAG are taken from environment variables:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -1063,6 +1161,9 @@ Some arguments in the example DAG are taken from OS environment variables:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -1123,6 +1224,9 @@ Example body defining the export operation:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -1217,6 +1321,9 @@ Example body defining the import operation:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -1289,6 +1396,9 @@ Example body defining the instance:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
@@ -1344,6 +1454,9 @@ Example body defining the instance:
 Using the operator
 """"""""""""""""""
 
+You can create the operator with or without project id. If project id is missing
+it will be retrieved from the service account used. Both variants are shown:
+
 .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py
     :language: python
     :dedent: 4
diff --git a/tests/contrib/hooks/test_gcp_container_hook.py b/tests/contrib/hooks/test_gcp_container_hook.py
index f4a21da7fc..dfcc98cb5c 100644
--- a/tests/contrib/hooks/test_gcp_container_hook.py
+++ b/tests/contrib/hooks/test_gcp_container_hook.py
@@ -26,13 +26,13 @@
 
 TASK_ID = 'test-gke-cluster-operator'
 CLUSTER_NAME = 'test-cluster'
-TEST_PROJECT_ID = 'test-project'
-ZONE = 'test-zone'
+TEST_GCP_PROJECT_ID = 'test-project'
+GKE_ZONE = 'test-zone'
 
 
 class GKEClusterHookDeleteTest(unittest.TestCase):
     def setUp(self):
-        self.gke_hook = GKEClusterHook(location=ZONE)
+        self.gke_hook = GKEClusterHook(location=GKE_ZONE)
         self.gke_hook._client = mock.Mock()
 
     @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto")
@@ -43,12 +43,15 @@ def test_delete_cluster(self, wait_mock, convert_mock):
 
         client_delete = self.gke_hook._client.delete_cluster = mock.Mock()
 
-        self.gke_hook.delete_cluster(name=CLUSTER_NAME, project_id=TEST_PROJECT_ID, retry=retry_mock,
+        self.gke_hook.delete_cluster(name=CLUSTER_NAME, project_id=TEST_GCP_PROJECT_ID,
+                                     retry=retry_mock,
                                      timeout=timeout_mock)
 
-        client_delete.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE,
+        client_delete.assert_called_with(project_id=TEST_GCP_PROJECT_ID,
+                                         zone=GKE_ZONE,
                                          cluster_id=CLUSTER_NAME,
-                                         retry=retry_mock, timeout=timeout_mock)
+                                         retry=retry_mock,
+                                         timeout=timeout_mock)
         wait_mock.assert_called_with(client_delete.return_value)
         convert_mock.assert_not_called()
 
@@ -63,7 +66,7 @@ def test_delete_cluster_not_found(self, wait_mock, convert_mock, log_mock):
         message = 'Not Found'
         self.gke_hook._client.delete_cluster.side_effect = NotFound(message=message)
 
-        self.gke_hook.delete_cluster(None)
+        self.gke_hook.delete_cluster('not-existing')
         wait_mock.assert_not_called()
         convert_mock.assert_not_called()
         log_mock.info.assert_any_call("Assuming Success: " + message)
@@ -76,14 +79,14 @@ def test_delete_cluster_error(self, wait_mock, convert_mock):
         self.gke_hook._client.delete_cluster.side_effect = AirflowException('400')
 
         with self.assertRaises(AirflowException):
-            self.gke_hook.delete_cluster(None)
+            self.gke_hook.delete_cluster('a-cluster')
             wait_mock.assert_not_called()
             convert_mock.assert_not_called()
 
 
 class GKEClusterHookCreateTest(unittest.TestCase):
     def setUp(self):
-        self.gke_hook = GKEClusterHook(location=ZONE)
+        self.gke_hook = GKEClusterHook(location=GKE_ZONE)
         self.gke_hook._client = mock.Mock()
 
     @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto")
@@ -99,10 +102,13 @@ def test_create_cluster_proto(self, wait_mock, convert_mock):
 
         client_create = self.gke_hook._client.create_cluster = mock.Mock()
 
-        self.gke_hook.create_cluster(mock_cluster_proto, project_id=TEST_PROJECT_ID, retry=retry_mock,
+        self.gke_hook.create_cluster(mock_cluster_proto,
+                                     project_id=TEST_GCP_PROJECT_ID,
+                                     retry=retry_mock,
                                      timeout=timeout_mock)
 
-        client_create.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE,
+        client_create.assert_called_with(project_id=TEST_GCP_PROJECT_ID,
+                                         zone=GKE_ZONE,
                                          cluster=mock_cluster_proto,
                                          retry=retry_mock, timeout=timeout_mock)
         wait_mock.assert_called_with(client_create.return_value)
@@ -118,10 +124,13 @@ def test_create_cluster_dict(self, wait_mock, convert_mock):
         client_create = self.gke_hook._client.create_cluster = mock.Mock()
         proto_mock = convert_mock.return_value = mock.Mock()
 
-        self.gke_hook.create_cluster(mock_cluster_dict, project_id=TEST_PROJECT_ID, retry=retry_mock,
+        self.gke_hook.create_cluster(mock_cluster_dict,
+                                     project_id=TEST_GCP_PROJECT_ID,
+                                     retry=retry_mock,
                                      timeout=timeout_mock)
 
-        client_create.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE,
+        client_create.assert_called_with(project_id=TEST_GCP_PROJECT_ID,
+                                         zone=GKE_ZONE,
                                          cluster=proto_mock,
                                          retry=retry_mock, timeout=timeout_mock)
         wait_mock.assert_called_with(client_create.return_value)
@@ -158,7 +167,7 @@ def test_create_cluster_already_exists(self, wait_mock, convert_mock, log_mock):
 
 class GKEClusterHookGetTest(unittest.TestCase):
     def setUp(self):
-        self.gke_hook = GKEClusterHook(location=ZONE)
+        self.gke_hook = GKEClusterHook(location=GKE_ZONE)
         self.gke_hook._client = mock.Mock()
 
     def test_get_cluster(self):
@@ -166,10 +175,13 @@ def test_get_cluster(self):
 
         client_get = self.gke_hook._client.get_cluster = mock.Mock()
 
-        self.gke_hook.get_cluster(name=CLUSTER_NAME, project_id=TEST_PROJECT_ID, retry=retry_mock,
+        self.gke_hook.get_cluster(name=CLUSTER_NAME,
+                                  project_id=TEST_GCP_PROJECT_ID,
+                                  retry=retry_mock,
                                   timeout=timeout_mock)
 
-        client_get.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE,
+        client_get.assert_called_with(project_id=TEST_GCP_PROJECT_ID,
+                                      zone=GKE_ZONE,
                                       cluster_id=CLUSTER_NAME,
                                       retry=retry_mock, timeout=timeout_mock)
 
@@ -177,10 +189,11 @@ def test_get_cluster(self):
 class GKEClusterHookTest(unittest.TestCase):
 
     def setUp(self):
-        self.gke_hook = GKEClusterHook(location=ZONE)
+        self.gke_hook = GKEClusterHook(location=GKE_ZONE)
         self.gke_hook._client = mock.Mock()
 
-    @mock.patch('airflow.contrib.hooks.gcp_container_hook.container_v1.ClusterManagerClient')
+    @mock.patch('airflow.contrib.hooks.gcp_container_hook.container_v1.'
+                'ClusterManagerClient')
     @mock.patch('airflow.contrib.hooks.gcp_container_hook.ClientInfo')
     @mock.patch('airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._get_credentials')
     def test_get_client(self, mock_get_credentials, mock_client_info, mock_client):
@@ -193,10 +206,9 @@ def test_get_client(self, mock_get_credentials, mock_client_info, mock_client):
 
     def test_get_operation(self):
         self.gke_hook._client.get_operation = mock.Mock()
-        self.gke_hook.get_operation('TEST_OP', project_id=TEST_PROJECT_ID)
-        self.gke_hook._client.get_operation.assert_called_with(project_id=TEST_PROJECT_ID,
-                                                               zone=ZONE,
-                                                               operation_id='TEST_OP')
+        self.gke_hook.get_operation('TEST_OP', project_id=TEST_GCP_PROJECT_ID)
+        self.gke_hook._client.get_operation.assert_called_with(
+            project_id=TEST_GCP_PROJECT_ID, zone=GKE_ZONE, operation_id='TEST_OP')
 
     def test_append_label(self):
         key = 'test-key'
@@ -244,11 +256,11 @@ def test_wait_for_response_running(self, time_mock, operation_mock):
 
         # Status goes from Running -> Pending -> Done
         operation_mock.side_effect = [pending_op, done_op]
-        self.gke_hook.wait_for_operation(running_op, project_id=TEST_PROJECT_ID)
+        self.gke_hook.wait_for_operation(running_op, project_id=TEST_GCP_PROJECT_ID)
 
         self.assertEqual(time_mock.call_count, 3)
-        operation_mock.assert_any_call(running_op.name, project_id=TEST_PROJECT_ID)
-        operation_mock.assert_any_call(pending_op.name, project_id=TEST_PROJECT_ID)
+        operation_mock.assert_any_call(running_op.name, project_id=TEST_GCP_PROJECT_ID)
+        operation_mock.assert_any_call(pending_op.name, project_id=TEST_GCP_PROJECT_ID)
         self.assertEqual(operation_mock.call_count, 2)
 
     @mock.patch("google.protobuf.json_format.Parse")
diff --git a/tests/contrib/hooks/test_gcp_dataproc_hook.py b/tests/contrib/hooks/test_gcp_dataproc_hook.py
index e22b27a8e7..acc4922f82 100644
--- a/tests/contrib/hooks/test_gcp_dataproc_hook.py
+++ b/tests/contrib/hooks/test_gcp_dataproc_hook.py
@@ -31,8 +31,8 @@
         mock = None
 
 JOB = 'test-job'
-PROJECT_ID = 'test-project-id'
-REGION = 'global'
+GCP_PROJECT_ID = 'test-project-id'
+GCP_REGION = 'global'
 TASK_ID = 'test-task-id'
 
 BASE_STRING = 'airflow.contrib.hooks.gcp_api_base_hook.{}'
@@ -53,8 +53,8 @@ def setUp(self):
     def test_submit(self, job_mock):
         with mock.patch(DATAPROC_STRING.format('DataProcHook.get_conn',
                                                return_value=None)):
-            self.dataproc_hook.submit(PROJECT_ID, JOB)
-            job_mock.assert_called_once_with(mock.ANY, PROJECT_ID, JOB, REGION,
+            self.dataproc_hook.submit(GCP_PROJECT_ID, JOB)
+            job_mock.assert_called_once_with(mock.ANY, GCP_PROJECT_ID, JOB, GCP_REGION,
                                              job_error_states=mock.ANY)
 
 
diff --git a/tests/contrib/hooks/test_gcp_sql_hook.py b/tests/contrib/hooks/test_gcp_sql_hook.py
index cb56237736..3d741c69f4 100644
--- a/tests/contrib/hooks/test_gcp_sql_hook.py
+++ b/tests/contrib/hooks/test_gcp_sql_hook.py
@@ -36,8 +36,8 @@
 class TestGcpSqlHook(unittest.TestCase):
     def test_instance_import_ex(self):
         # Mocking __init__ with an empty anonymous function
-        with mock.patch.object(CloudSqlHook, "__init__", lambda x, y, z: None):
-            hook = CloudSqlHook(None, None)
+        with mock.patch.object(CloudSqlHook, '__init__', lambda x: None):
+            hook = CloudSqlHook()
             # Simulating HttpError inside import_instance
             hook.get_conn = mock.Mock(
                 side_effect=HttpError(resp={'status': '400'},
@@ -50,8 +50,8 @@ def test_instance_import_ex(self):
 
     def test_instance_export_ex(self):
         # Mocking __init__ with an empty anonymous function
-        with mock.patch.object(CloudSqlHook, "__init__", lambda x, y, z: None):
-            hook = CloudSqlHook(None, None)
+        with mock.patch.object(CloudSqlHook, '__init__', lambda x: None):
+            hook = CloudSqlHook()
             # Simulating HttpError inside export_instance
             hook.get_conn = mock.Mock(
                 side_effect=HttpError(resp={'status': '400'},
diff --git a/tests/contrib/operators/postgres_local_executor.cfg b/tests/contrib/operators/postgres_local_executor.cfg
new file mode 100644
index 0000000000..1f8eb3d3c3
--- /dev/null
+++ b/tests/contrib/operators/postgres_local_executor.cfg
@@ -0,0 +1,28 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+
+
+# This configuration file is used to override
+# the executor to be LocalExecutor and switch to using Postgres as the database
+# Some system tests require LocalExecutor to be used because of parallelism required
+# One example is test_gcp_bigtable_operator_system.py in which sensor
+# Has to be run in parallell with a task to create BigTable table
+[core]
+executor = LocalExecutor
+sql_alchemy_conn = postgresql:///airflow/airflow.db
diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py
index 84f1750bed..3046994109 100644
--- a/tests/contrib/operators/test_bigquery_operator.py
+++ b/tests/contrib/operators/test_bigquery_operator.py
@@ -41,7 +41,7 @@
 
 TASK_ID = 'test-bq-create-table-operator'
 TEST_DATASET = 'test-dataset'
-TEST_PROJECT_ID = 'test-project'
+TEST_GCP_PROJECT_ID = 'test-project'
 TEST_TABLE_ID = 'test-table-id'
 TEST_GCS_BUCKET = 'test-bucket'
 TEST_GCS_DATA = ['dir1/*.csv']
@@ -56,7 +56,7 @@ class BigQueryCreateEmptyTableOperatorTest(unittest.TestCase):
     def test_execute(self, mock_hook):
         operator = BigQueryCreateEmptyTableOperator(task_id=TASK_ID,
                                                     dataset_id=TEST_DATASET,
-                                                    project_id=TEST_PROJECT_ID,
+                                                    project_id=TEST_GCP_PROJECT_ID,
                                                     table_id=TEST_TABLE_ID)
 
         operator.execute(None)
@@ -66,7 +66,7 @@ def test_execute(self, mock_hook):
             .create_empty_table \
             .assert_called_once_with(
                 dataset_id=TEST_DATASET,
-                project_id=TEST_PROJECT_ID,
+                project_id=TEST_GCP_PROJECT_ID,
                 table_id=TEST_TABLE_ID,
                 schema_fields=None,
                 time_partitioning={},
@@ -120,7 +120,7 @@ def test_execute(self, mock_hook):
         operator = BigQueryDeleteDatasetOperator(
             task_id=TASK_ID,
             dataset_id=TEST_DATASET,
-            project_id=TEST_PROJECT_ID
+            project_id=TEST_GCP_PROJECT_ID
         )
 
         operator.execute(None)
@@ -130,7 +130,7 @@ def test_execute(self, mock_hook):
             .delete_dataset \
             .assert_called_once_with(
                 dataset_id=TEST_DATASET,
-                project_id=TEST_PROJECT_ID
+                project_id=TEST_GCP_PROJECT_ID
             )
 
 
@@ -140,7 +140,7 @@ def test_execute(self, mock_hook):
         operator = BigQueryCreateEmptyDatasetOperator(
             task_id=TASK_ID,
             dataset_id=TEST_DATASET,
-            project_id=TEST_PROJECT_ID
+            project_id=TEST_GCP_PROJECT_ID
         )
 
         operator.execute(None)
@@ -150,7 +150,7 @@ def test_execute(self, mock_hook):
             .create_empty_dataset \
             .assert_called_once_with(
                 dataset_id=TEST_DATASET,
-                project_id=TEST_PROJECT_ID,
+                project_id=TEST_GCP_PROJECT_ID,
                 dataset_reference={}
             )
 
diff --git a/tests/contrib/operators/test_dataproc_operator.py b/tests/contrib/operators/test_dataproc_operator.py
index 219d7255ff..28e4e035e9 100644
--- a/tests/contrib/operators/test_dataproc_operator.py
+++ b/tests/contrib/operators/test_dataproc_operator.py
@@ -50,9 +50,9 @@
 
 TASK_ID = 'test-dataproc-operator'
 CLUSTER_NAME = 'test-cluster-name'
-PROJECT_ID = 'test-project-id'
+GCP_PROJECT_ID = 'test-project-id'
 NUM_WORKERS = 123
-ZONE = 'us-central1-a'
+GCE_ZONE = 'us-central1-a'
 NETWORK_URI = '/projects/project_id/regions/global/net'
 SUBNETWORK_URI = '/projects/project_id/regions/global/subnet'
 INTERNAL_IP_ONLY = True
@@ -78,7 +78,7 @@
 AUTO_DELETE_TIME = datetime.datetime(2017, 6, 7)
 AUTO_DELETE_TTL = 654
 DEFAULT_DATE = datetime.datetime(2017, 6, 6)
-REGION = 'test-region'
+GCP_REGION = 'test-region'
 MAIN_URI = 'test-uri'
 TEMPLATE_ID = 'template-id'
 
@@ -87,7 +87,7 @@
 DATAPROC_JOB_TO_SUBMIT = {
     'job': {
         'reference': {
-            'projectId': PROJECT_ID,
+            'projectId': GCP_PROJECT_ID,
             'jobId': DATAPROC_JOB_ID,
         },
         'placement': {
@@ -118,9 +118,9 @@ def setUp(self):
                 DataprocClusterCreateOperator(
                     task_id=TASK_ID,
                     cluster_name=CLUSTER_NAME,
-                    project_id=PROJECT_ID,
+                    project_id=GCP_PROJECT_ID,
                     num_workers=NUM_WORKERS,
-                    zone=ZONE,
+                    zone=GCE_ZONE,
                     network_uri=NETWORK_URI,
                     subnetwork_uri=SUBNETWORK_URI,
                     internal_ip_only=INTERNAL_IP_ONLY,
@@ -154,9 +154,9 @@ def test_init(self):
         """Test DataProcClusterOperator instance is properly initialized."""
         for suffix, dataproc_operator in enumerate(self.dataproc_operators):
             self.assertEqual(dataproc_operator.cluster_name, CLUSTER_NAME)
-            self.assertEqual(dataproc_operator.project_id, PROJECT_ID)
+            self.assertEqual(dataproc_operator.project_id, GCP_PROJECT_ID)
             self.assertEqual(dataproc_operator.num_workers, NUM_WORKERS)
-            self.assertEqual(dataproc_operator.zone, ZONE)
+            self.assertEqual(dataproc_operator.zone, GCE_ZONE)
             self.assertEqual(dataproc_operator.network_uri, NETWORK_URI)
             self.assertEqual(dataproc_operator.subnetwork_uri, SUBNETWORK_URI)
             self.assertEqual(dataproc_operator.tags, TAGS)
@@ -186,7 +186,7 @@ def test_build_cluster_data(self):
         for suffix, dataproc_operator in enumerate(self.dataproc_operators):
             cluster_data = dataproc_operator._build_cluster_data()
             self.assertEqual(cluster_data['clusterName'], CLUSTER_NAME)
-            self.assertEqual(cluster_data['projectId'], PROJECT_ID)
+            self.assertEqual(cluster_data['projectId'], GCP_PROJECT_ID)
             self.assertEqual(cluster_data['config']['softwareConfig'],
                              {'imageVersion': IMAGE_VERSION})
             self.assertEqual(cluster_data['config']['configBucket'], STORAGE_BUCKET)
@@ -223,9 +223,9 @@ def test_build_cluster_data_with_autoDeleteTime(self):
         dataproc_operator = DataprocClusterCreateOperator(
             task_id=TASK_ID,
             cluster_name=CLUSTER_NAME,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             num_workers=NUM_WORKERS,
-            zone=ZONE,
+            zone=GCE_ZONE,
             dag=self.dag,
             auto_delete_time=AUTO_DELETE_TIME,
         )
@@ -237,9 +237,9 @@ def test_build_cluster_data_with_autoDeleteTtl(self):
         dataproc_operator = DataprocClusterCreateOperator(
             task_id=TASK_ID,
             cluster_name=CLUSTER_NAME,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             num_workers=NUM_WORKERS,
-            zone=ZONE,
+            zone=GCE_ZONE,
             dag=self.dag,
             auto_delete_ttl=AUTO_DELETE_TTL,
         )
@@ -251,9 +251,9 @@ def test_build_cluster_data_with_autoDeleteTime_and_autoDeleteTtl(self):
         dataproc_operator = DataprocClusterCreateOperator(
             task_id=TASK_ID,
             cluster_name=CLUSTER_NAME,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             num_workers=NUM_WORKERS,
-            zone=ZONE,
+            zone=GCE_ZONE,
             dag=self.dag,
             auto_delete_time=AUTO_DELETE_TIME,
             auto_delete_ttl=AUTO_DELETE_TTL,
@@ -270,9 +270,9 @@ def test_init_with_image_version_and_custom_image_both_set(self):
             DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=NUM_WORKERS,
-                zone=ZONE,
+                zone=GCE_ZONE,
                 dag=self.dag,
                 image_version=IMAGE_VERSION,
                 custom_image=CUSTOM_IMAGE
@@ -282,9 +282,9 @@ def test_init_with_custom_image(self):
         dataproc_operator = DataprocClusterCreateOperator(
             task_id=TASK_ID,
             cluster_name=CLUSTER_NAME,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             num_workers=NUM_WORKERS,
-            zone=ZONE,
+            zone=GCE_ZONE,
             dag=self.dag,
             custom_image=CUSTOM_IMAGE
         )
@@ -292,7 +292,7 @@ def test_init_with_custom_image(self):
         cluster_data = dataproc_operator._build_cluster_data()
         expected_custom_image_url = \
             'https://www.googleapis.com/compute/beta/projects/' \
-            '{}/global/images/{}'.format(PROJECT_ID, CUSTOM_IMAGE)
+            '{}/global/images/{}'.format(GCP_PROJECT_ID, CUSTOM_IMAGE)
         self.assertEqual(cluster_data['config']['masterConfig']['imageUri'],
                          expected_custom_image_url)
         self.assertEqual(cluster_data['config']['workerConfig']['imageUri'],
@@ -302,10 +302,10 @@ def test_build_single_node_cluster(self):
         dataproc_operator = DataprocClusterCreateOperator(
             task_id=TASK_ID,
             cluster_name=CLUSTER_NAME,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             num_workers=0,
             num_preemptible_workers=0,
-            zone=ZONE,
+            zone=GCE_ZONE,
             dag=self.dag
         )
         cluster_data = dataproc_operator._build_cluster_data()
@@ -318,10 +318,10 @@ def test_init_cluster_with_zero_workers_and_not_non_zero_preemtibles(self):
             DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=0,
                 num_preemptible_workers=2,
-                zone=ZONE,
+                zone=GCE_ZONE,
                 dag=self.dag,
                 image_version=IMAGE_VERSION,
             )
@@ -332,9 +332,9 @@ def test_cluster_name_log_no_sub(self):
             dataproc_task = DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=NUM_WORKERS,
-                zone=ZONE,
+                zone=GCE_ZONE,
                 dag=self.dag
             )
             with patch.object(dataproc_task.log, 'info') as mock_info:
@@ -348,9 +348,9 @@ def test_cluster_name_log_sub(self):
             dataproc_task = DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name='smoke-cluster-{{ ts_nodash }}',
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=NUM_WORKERS,
-                zone=ZONE,
+                zone=GCE_ZONE,
                 dag=self.dag
             )
             with patch.object(dataproc_task.log, 'info') as mock_info:
@@ -371,9 +371,9 @@ def create_cluster_with_invalid_internal_ip_only_setup():
             create_cluster = DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=NUM_WORKERS,
-                zone=ZONE,
+                zone=GCE_ZONE,
                 dag=self.dag,
                 internal_ip_only=True)
 
@@ -418,7 +418,7 @@ def test_cluster_name_log_no_sub(self):
             dataproc_task = DataprocClusterScaleOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=NUM_WORKERS,
                 num_preemptible_workers=NUM_PREEMPTIBLE_WORKERS,
                 dag=self.dag
@@ -434,7 +434,7 @@ def test_cluster_name_log_sub(self):
             dataproc_task = DataprocClusterScaleOperator(
                 task_id=TASK_ID,
                 cluster_name='smoke-cluster-{{ ts_nodash }}',
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 num_workers=NUM_WORKERS,
                 num_preemptible_workers=NUM_PREEMPTIBLE_WORKERS,
                 dag=self.dag
@@ -482,7 +482,7 @@ def test_cluster_name_log_no_sub(self):
             dataproc_task = DataprocClusterDeleteOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 dag=self.dag
             )
             with patch.object(dataproc_task.log, 'info') as mock_info:
@@ -496,7 +496,7 @@ def test_cluster_name_log_sub(self):
             dataproc_task = DataprocClusterDeleteOperator(
                 task_id=TASK_ID,
                 cluster_name='smoke-cluster-{{ ts_nodash }}',
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 dag=self.dag
             )
 
@@ -520,12 +520,12 @@ def test_hook_correct_region():
         with patch(HOOK) as mock_hook:
             dataproc_task = DataProcHadoopOperator(
                 task_id=TASK_ID,
-                region=REGION
+                region=GCP_REGION
             )
 
             dataproc_task.execute(None)
             mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY,
-                                                                  REGION, mock.ANY)
+                                                                  GCP_REGION, mock.ANY)
 
     @staticmethod
     def test_dataproc_job_id_is_set():
@@ -544,12 +544,12 @@ def test_hook_correct_region():
         with patch(HOOK) as mock_hook:
             dataproc_task = DataProcHiveOperator(
                 task_id=TASK_ID,
-                region=REGION
+                region=GCP_REGION
             )
 
             dataproc_task.execute(None)
             mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY,
-                                                                  REGION, mock.ANY)
+                                                                  GCP_REGION, mock.ANY)
 
     @staticmethod
     def test_dataproc_job_id_is_set():
@@ -569,12 +569,12 @@ def test_hook_correct_region():
             dataproc_task = DataProcPySparkOperator(
                 task_id=TASK_ID,
                 main=MAIN_URI,
-                region=REGION
+                region=GCP_REGION
             )
 
             dataproc_task.execute(None)
             mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY,
-                                                                  REGION, mock.ANY)
+                                                                  GCP_REGION, mock.ANY)
 
     @staticmethod
     def test_dataproc_job_id_is_set():
@@ -594,12 +594,12 @@ def test_hook_correct_region():
         with patch(HOOK) as mock_hook:
             dataproc_task = DataProcSparkOperator(
                 task_id=TASK_ID,
-                region=REGION
+                region=GCP_REGION
             )
 
             dataproc_task.execute(None)
             mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY,
-                                                                  REGION, mock.ANY)
+                                                                  GCP_REGION, mock.ANY)
 
     @staticmethod
     def test_dataproc_job_id_is_set():
@@ -642,8 +642,8 @@ def test_workflow(self):
 
             dataproc_task = DataprocWorkflowTemplateInstantiateOperator(
                 task_id=TASK_ID,
-                project_id=PROJECT_ID,
-                region=REGION,
+                project_id=GCP_PROJECT_ID,
+                region=GCP_REGION,
                 template_id=TEMPLATE_ID,
                 dag=self.dag
             )
@@ -694,7 +694,7 @@ def test_iniline_workflow(self):
                         "cluster_name": CLUSTER_NAME,
                         "config": {
                             "gce_cluster_config": {
-                                "zone_uri": ZONE,
+                                "zone_uri": GCE_ZONE,
                             }
                         }
                     }
@@ -710,8 +710,8 @@ def test_iniline_workflow(self):
 
             dataproc_task = DataprocWorkflowTemplateInstantiateInlineOperator(
                 task_id=TASK_ID,
-                project_id=PROJECT_ID,
-                region=REGION,
+                project_id=GCP_PROJECT_ID,
+                region=GCP_REGION,
                 template=template,
                 dag=self.dag
             )
diff --git a/tests/contrib/operators/test_gcp_base.py b/tests/contrib/operators/test_gcp_base.py
deleted file mode 100644
index 2df3e39ebd..0000000000
--- a/tests/contrib/operators/test_gcp_base.py
+++ /dev/null
@@ -1,176 +0,0 @@
-# -*- coding: utf-8 -*-
-#
-# 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 json
-import os
-import subprocess
-import unittest
-
-from airflow import models, settings, configuration, AirflowException
-from airflow.models.connection import Connection
-from airflow.utils.timezone import datetime
-
-DEFAULT_DATE = datetime(2015, 1, 1)
-
-KEYPATH_EXTRA = 'extra__google_cloud_platform__key_path'
-KEYFILE_DICT_EXTRA = 'extra__google_cloud_platform__keyfile_dict'
-SCOPE_EXTRA = 'extra__google_cloud_platform__scope'
-PROJECT_EXTRA = 'extra__google_cloud_platform__project'
-
-AIRFLOW_MAIN_FOLDER = os.path.realpath(os.path.join(
-    os.path.dirname(os.path.realpath(__file__)),
-    os.pardir, os.pardir, os.pardir))
-
-CONTRIB_OPERATORS_EXAMPLES_DAG_FOLDER = os.path.join(
-    AIRFLOW_MAIN_FOLDER, "airflow", "contrib", "example_dags")
-
-OPERATORS_EXAMPLES_DAG_FOLDER = os.path.join(
-    AIRFLOW_MAIN_FOLDER, "airflow", "example_dags")
-
-TESTS_DAG_FOLDER = os.path.join(
-    AIRFLOW_MAIN_FOLDER, "tests", "dags")
-
-GCP_FOLDER_ENVIRONMENT_VARIABLE = "GCP_SERVICE_ACCOUNT_KEY_FOLDER"
-
-GCP_COMPUTE_KEY = 'gcp_compute.json'
-GCP_FUNCTION_KEY = 'gcp_function.json'
-GCP_CLOUDSQL_KEY = 'gcp_cloudsql.json'
-GCP_BIGTABLE_KEY = 'gcp_bigtable.json'
-GCP_SPANNER_KEY = 'gcp_spanner.json'
-GCP_GCS_KEY = 'gcp_gcs.json'
-
-SKIP_TEST_WARNING = """
-The test is only run when there is GCP connection available! "
-Set GCP_SERVICE_ACCOUNT_KEY_FOLDER environment variable if "
-you want to run them".
-"""
-
-
-class BaseGcpIntegrationTestCase(unittest.TestCase):
-    def __init__(self,
-                 method_name,
-                 dag_id,
-                 gcp_key,
-                 dag_name=None,
-                 example_dags_folder=CONTRIB_OPERATORS_EXAMPLES_DAG_FOLDER,
-                 project_extra=None):
-        super(BaseGcpIntegrationTestCase, self).__init__(method_name)
-        self.dag_id = dag_id
-        self.dag_name = self.dag_id + '.py' if not dag_name else dag_name
-        self.gcp_key = gcp_key
-        self.example_dags_folder = example_dags_folder
-        self.project_extra = project_extra
-        self.full_key_path = None
-
-    def _gcp_authenticate(self):
-        key_dir_path = os.environ['GCP_SERVICE_ACCOUNT_KEY_FOLDER']
-        self.full_key_path = os.path.join(key_dir_path, self.gcp_key)
-
-        if not os.path.isfile(self.full_key_path):
-            raise Exception("The key {} could not be found. Please copy it to the "
-                            "{} folder.".format(self.gcp_key, key_dir_path))
-        print("Setting the GCP key to {}".format(self.full_key_path))
-        # Checking if we can authenticate using service account credentials provided
-        retcode = subprocess.call(['gcloud', 'auth', 'activate-service-account',
-                                   '--key-file={}'.format(self.full_key_path)])
-        if retcode != 0:
-            raise AirflowException("The gcloud auth method was not successful!")
-        self.update_connection_with_key_path()
-        # Now we revoke all authentication here because we want to make sure
-        # that all works fine with the credentials retrieved from the gcp_connection
-        subprocess.call(['gcloud', 'auth', 'revoke'])
-
-    def update_connection_with_key_path(self):
-        session = settings.Session()
-        try:
-            conn = session.query(Connection).filter(
-                Connection.conn_id == 'google_cloud_default')[0]
-            extras = conn.extra_dejson
-            extras[KEYPATH_EXTRA] = self.full_key_path
-            if extras.get(KEYFILE_DICT_EXTRA):
-                del extras[KEYFILE_DICT_EXTRA]
-            extras[SCOPE_EXTRA] = 'https://www.googleapis.com/auth/cloud-platform'
-            extras[PROJECT_EXTRA] = self.project_extra
-            conn.extra = json.dumps(extras)
-            session.commit()
-        except BaseException as e:
-            print('Airflow DB Session error:' + str(e.message))
-            session.rollback()
-            raise
-        finally:
-            session.close()
-
-    def update_connection_with_dictionary(self):
-        session = settings.Session()
-        try:
-            conn = session.query(Connection).filter(
-                Connection.conn_id == 'google_cloud_default')[0]
-            extras = conn.extra_dejson
-            with open(self.full_key_path, "r") as f:
-                content = json.load(f)
-            extras[KEYFILE_DICT_EXTRA] = json.dumps(content)
-            if extras.get(KEYPATH_EXTRA):
-                del extras[KEYPATH_EXTRA]
-            extras[SCOPE_EXTRA] = 'https://www.googleapis.com/auth/cloud-platform'
-            extras[PROJECT_EXTRA] = self.project_extra
-            conn.extra = json.dumps(extras)
-            session.commit()
-        except BaseException as e:
-            print('Airflow DB Session error:' + str(e.message))
-            session.rollback()
-            raise
-        finally:
-            session.close()
-
-    def _symlink_dag(self):
-        target_path = os.path.join(TESTS_DAG_FOLDER, self.dag_name)
-        if os.path.exists(target_path):
-            os.remove(target_path)
-        os.symlink(
-            os.path.join(self.example_dags_folder, self.dag_name),
-            os.path.join(target_path))
-
-    def _rm_symlink_dag(self):
-        os.remove(os.path.join(TESTS_DAG_FOLDER, self.dag_name))
-
-    def _run_dag(self):
-        dag_bag = models.DagBag(dag_folder=TESTS_DAG_FOLDER, include_examples=False)
-        self.args = {'owner': 'airflow', 'start_date': DEFAULT_DATE}
-        dag = dag_bag.get_dag(self.dag_id)
-        dag.clear(reset_dag_runs=True)
-        dag.run(ignore_first_depends_on_past=True, verbose=True)
-
-    def setUp(self):
-        configuration.conf.load_test_config()
-        self._gcp_authenticate()
-        self._symlink_dag()
-
-    def tearDown(self):
-        self._rm_symlink_dag()
-
-    @staticmethod
-    def skip_check(key):
-        if GCP_FOLDER_ENVIRONMENT_VARIABLE not in os.environ:
-            return True
-        key_folder = os.environ[GCP_FOLDER_ENVIRONMENT_VARIABLE]
-        if not os.path.isdir(key_folder):
-            return True
-        key_path = os.path.join(key_folder, key)
-        if not os.path.isfile(key_path):
-            return True
-        return False
diff --git a/tests/contrib/operators/test_gcp_bigtable_operator.py b/tests/contrib/operators/test_gcp_bigtable_operator.py
index 8417efc997..300bc14b30 100644
--- a/tests/contrib/operators/test_gcp_bigtable_operator.py
+++ b/tests/contrib/operators/test_gcp_bigtable_operator.py
@@ -25,9 +25,13 @@
 from parameterized import parameterized
 
 from airflow import AirflowException
-from airflow.contrib.operators.gcp_bigtable_operator import BigtableInstanceDeleteOperator, \
-    BigtableTableDeleteOperator, BigtableTableCreateOperator, BigtableTableWaitForReplicationSensor, \
-    BigtableClusterUpdateOperator, BigtableInstanceCreateOperator
+from airflow.contrib.operators.gcp_bigtable_operator import \
+    BigtableInstanceDeleteOperator, \
+    BigtableTableDeleteOperator, \
+    BigtableTableCreateOperator, \
+    BigtableTableWaitForReplicationSensor, \
+    BigtableClusterUpdateOperator, \
+    BigtableInstanceCreateOperator
 
 try:
     # noinspection PyProtectedMember
@@ -50,13 +54,13 @@
 
 class BigtableInstanceCreateTest(unittest.TestCase):
     @parameterized.expand([
-        ('project_id', '', INSTANCE_ID, CLUSTER_ID, CLUSTER_ZONE),
         ('instance_id', PROJECT_ID, '', CLUSTER_ID, CLUSTER_ZONE),
         ('main_cluster_id', PROJECT_ID, INSTANCE_ID, '', CLUSTER_ZONE),
         ('main_cluster_zone', PROJECT_ID, INSTANCE_ID, CLUSTER_ID, ''),
     ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0])
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_empty_attribute(self, missing_attribute, project_id, instance_id, main_cluster_id,
+    def test_empty_attribute(self, missing_attribute, project_id, instance_id,
+                             main_cluster_id,
                              main_cluster_zone, mock_hook):
         with self.assertRaises(AirflowException) as e:
             BigtableInstanceCreateOperator(
@@ -86,6 +90,21 @@ def test_create_instance_that_exists(self, mock_hook):
         mock_hook.assert_called_once_with()
         mock_hook.return_value.create_instance.assert_not_called()
 
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_create_instance_that_exists_empty_project_id(self, mock_hook):
+        mock_hook.return_value.get_instance.return_value = mock.Mock(Instance)
+
+        op = BigtableInstanceCreateOperator(
+            instance_id=INSTANCE_ID,
+            main_cluster_id=CLUSTER_ID,
+            main_cluster_zone=CLUSTER_ZONE,
+            task_id="id"
+        )
+        op.execute(None)
+
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.create_instance.assert_not_called()
+
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
     def test_different_error_reraised(self, mock_hook):
         mock_hook.return_value.get_instance.return_value = None
@@ -105,19 +124,20 @@ def test_different_error_reraised(self, mock_hook):
 
         mock_hook.assert_called_once_with()
         mock_hook.return_value.create_instance.assert_called_once_with(
-            PROJECT_ID, INSTANCE_ID, CLUSTER_ID, CLUSTER_ZONE, None, None, None, None, None, None, None, None
+            PROJECT_ID, INSTANCE_ID, CLUSTER_ID, CLUSTER_ZONE, None, None, None, None,
+            None, None, None, None
         )
 
 
 class BigtableClusterUpdateTest(unittest.TestCase):
     @parameterized.expand([
-        ('project_id', '', INSTANCE_ID, CLUSTER_ID, NODES),
         ('instance_id', PROJECT_ID, '', CLUSTER_ID, NODES),
         ('cluster_id', PROJECT_ID, INSTANCE_ID, '', NODES),
         ('nodes', PROJECT_ID, INSTANCE_ID, CLUSTER_ID, ''),
     ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0])
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_empty_attribute(self, missing_attribute, project_id, instance_id, cluster_id, nodes, mock_hook):
+    def test_empty_attribute(self, missing_attribute, project_id, instance_id,
+                             cluster_id, nodes, mock_hook):
         with self.assertRaises(AirflowException) as e:
             BigtableClusterUpdateOperator(
                 project_id=project_id,
@@ -145,7 +165,28 @@ def test_updating_cluster_but_instance_does_not_exists(self, mock_hook):
             op.execute(None)
 
         err = e.exception
-        self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(INSTANCE_ID))
+        self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(
+            INSTANCE_ID))
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.update_cluster.assert_not_called()
+
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_updating_cluster_but_instance_does_not_exists_empty_project_id(self,
+                                                                            mock_hook):
+        mock_hook.return_value.get_instance.return_value = None
+
+        with self.assertRaises(AirflowException) as e:
+            op = BigtableClusterUpdateOperator(
+                instance_id=INSTANCE_ID,
+                cluster_id=CLUSTER_ID,
+                nodes=NODES,
+                task_id="id"
+            )
+            op.execute(None)
+
+        err = e.exception
+        self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(
+            INSTANCE_ID))
         mock_hook.assert_called_once_with()
         mock_hook.return_value.update_cluster.assert_not_called()
 
@@ -168,10 +209,37 @@ def test_updating_cluster_that_does_not_exists(self, mock_hook):
         err = e.exception
         self.assertEqual(
             str(err),
-            "Dependency: cluster '{}' does not exist for instance '{}'.".format(CLUSTER_ID, INSTANCE_ID)
+            "Dependency: cluster '{}' does not exist for instance '{}'.".format(
+                CLUSTER_ID, INSTANCE_ID)
+        )
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.update_cluster.assert_called_once_with(instance,
+                                                                      CLUSTER_ID, NODES)
+
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_updating_cluster_that_does_not_exists_empty_project_id(self, mock_hook):
+        instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance)
+        mock_hook.return_value.update_cluster.side_effect = mock.Mock(
+            side_effect=google.api_core.exceptions.NotFound("Cluster not found."))
+
+        with self.assertRaises(AirflowException) as e:
+            op = BigtableClusterUpdateOperator(
+                instance_id=INSTANCE_ID,
+                cluster_id=CLUSTER_ID,
+                nodes=NODES,
+                task_id="id"
+            )
+            op.execute(None)
+
+        err = e.exception
+        self.assertEqual(
+            str(err),
+            "Dependency: cluster '{}' does not exist for instance '{}'.".format(
+                CLUSTER_ID, INSTANCE_ID)
         )
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.update_cluster.assert_called_once_with(instance, CLUSTER_ID, NODES)
+        mock_hook.return_value.update_cluster.assert_called_once_with(instance,
+                                                                      CLUSTER_ID, NODES)
 
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
     def test_different_error_reraised(self, mock_hook):
@@ -190,7 +258,8 @@ def test_different_error_reraised(self, mock_hook):
             op.execute(None)
 
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.update_cluster.assert_called_once_with(instance, CLUSTER_ID, NODES)
+        mock_hook.return_value.update_cluster.assert_called_once_with(instance,
+                                                                      CLUSTER_ID, NODES)
 
 
 class BigtableInstanceDeleteTest(unittest.TestCase):
@@ -203,10 +272,21 @@ def test_delete_execute(self, mock_hook):
         )
         op.execute(None)
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID, INSTANCE_ID)
+        mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID,
+                                                                       INSTANCE_ID)
+
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_delete_execute_empty_project_id(self, mock_hook):
+        op = BigtableInstanceDeleteOperator(
+            instance_id=INSTANCE_ID,
+            task_id="id"
+        )
+        op.execute(None)
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.delete_instance.assert_called_once_with(None,
+                                                                       INSTANCE_ID)
 
     @parameterized.expand([
-        ('project_id', '', INSTANCE_ID),
         ('instance_id', PROJECT_ID, ''),
     ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0])
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
@@ -232,7 +312,21 @@ def test_deleting_instance_that_doesnt_exists(self, mock_hook):
             side_effect=google.api_core.exceptions.NotFound("Instance not found."))
         op.execute(None)
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID, INSTANCE_ID)
+        mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID,
+                                                                       INSTANCE_ID)
+
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_deleting_instance_that_doesnt_exists_empty_project_id(self, mock_hook):
+        op = BigtableInstanceDeleteOperator(
+            instance_id=INSTANCE_ID,
+            task_id="id"
+        )
+        mock_hook.return_value.delete_instance.side_effect = mock.Mock(
+            side_effect=google.api_core.exceptions.NotFound("Instance not found."))
+        op.execute(None)
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.delete_instance.assert_called_once_with(None,
+                                                                       INSTANCE_ID)
 
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
     def test_different_error_reraised(self, mock_hook):
@@ -248,7 +342,8 @@ def test_different_error_reraised(self, mock_hook):
             op.execute(None)
 
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID, INSTANCE_ID)
+        mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID,
+                                                                       INSTANCE_ID)
 
 
 class BigtableTableDeleteTest(unittest.TestCase):
@@ -262,15 +357,16 @@ def test_delete_execute(self, mock_hook):
         )
         op.execute(None)
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID, INSTANCE_ID, TABLE_ID)
+        mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID,
+                                                                    INSTANCE_ID, TABLE_ID)
 
     @parameterized.expand([
-        ('project_id', '', INSTANCE_ID, TABLE_ID),
         ('instance_id', PROJECT_ID, '', TABLE_ID),
         ('table_id', PROJECT_ID, INSTANCE_ID, ''),
     ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0])
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id, mock_hook):
+    def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id,
+                             mock_hook):
         with self.assertRaises(AirflowException) as e:
             BigtableTableDeleteOperator(
                 project_id=project_id,
@@ -295,7 +391,23 @@ def test_deleting_table_that_doesnt_exists(self, mock_hook):
             side_effect=google.api_core.exceptions.NotFound("Table not found."))
         op.execute(None)
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID, INSTANCE_ID, TABLE_ID)
+        mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID,
+                                                                    INSTANCE_ID, TABLE_ID)
+
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_deleting_table_that_doesnt_exists_empty_project_id(self, mock_hook):
+        op = BigtableTableDeleteOperator(
+            instance_id=INSTANCE_ID,
+            table_id=TABLE_ID,
+            task_id="id"
+        )
+
+        mock_hook.return_value.delete_table.side_effect = mock.Mock(
+            side_effect=google.api_core.exceptions.NotFound("Table not found."))
+        op.execute(None)
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.delete_table.assert_called_once_with(None,
+                                                                    INSTANCE_ID, TABLE_ID)
 
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
     def test_deleting_table_when_instance_doesnt_exists(self, mock_hook):
@@ -310,7 +422,8 @@ def test_deleting_table_when_instance_doesnt_exists(self, mock_hook):
         with self.assertRaises(AirflowException) as e:
             op.execute(None)
         err = e.exception
-        self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(INSTANCE_ID))
+        self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(
+            INSTANCE_ID))
         mock_hook.assert_called_once_with()
         mock_hook.return_value.delete_table.assert_not_called()
 
@@ -329,7 +442,8 @@ def test_different_error_reraised(self, mock_hook):
             op.execute(None)
 
         mock_hook.assert_called_once_with()
-        mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID, INSTANCE_ID, TABLE_ID)
+        mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID,
+                                                                    INSTANCE_ID, TABLE_ID)
 
 
 class BigtableTableCreateTest(unittest.TestCase):
@@ -350,12 +464,12 @@ def test_create_execute(self, mock_hook):
             instance, TABLE_ID, INITIAL_SPLIT_KEYS, EMPTY_COLUMN_FAMILIES)
 
     @parameterized.expand([
-        ('project_id', '', INSTANCE_ID, TABLE_ID),
         ('instance_id', PROJECT_ID, '', TABLE_ID),
         ('table_id', PROJECT_ID, INSTANCE_ID, ''),
     ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0])
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id, mock_hook):
+    def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id,
+                             mock_hook):
         with self.assertRaises(AirflowException) as e:
             BigtableTableCreateOperator(
                 project_id=project_id,
@@ -383,7 +497,8 @@ def test_instance_not_exists(self, mock_hook):
         err = e.exception
         self.assertEqual(
             str(err),
-            "Dependency: instance '{}' does not exist in project '{}'.".format(INSTANCE_ID, PROJECT_ID)
+            "Dependency: instance '{}' does not exist in project '{}'.".format(
+                INSTANCE_ID, PROJECT_ID)
         )
         mock_hook.assert_called_once_with()
 
@@ -398,7 +513,29 @@ def test_creating_table_that_exists(self, mock_hook):
             task_id="id"
         )
 
-        mock_hook.return_value.get_column_families_for_table.return_value = EMPTY_COLUMN_FAMILIES
+        mock_hook.return_value.get_column_families_for_table.return_value = \
+            EMPTY_COLUMN_FAMILIES
+        instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance)
+        mock_hook.return_value.create_table.side_effect = mock.Mock(
+            side_effect=google.api_core.exceptions.AlreadyExists("Table already exists."))
+        op.execute(None)
+
+        mock_hook.assert_called_once_with()
+        mock_hook.return_value.create_table.assert_called_once_with(
+            instance, TABLE_ID, INITIAL_SPLIT_KEYS, EMPTY_COLUMN_FAMILIES)
+
+    @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
+    def test_creating_table_that_exists_empty_project_id(self, mock_hook):
+        op = BigtableTableCreateOperator(
+            instance_id=INSTANCE_ID,
+            table_id=TABLE_ID,
+            initial_split_keys=INITIAL_SPLIT_KEYS,
+            column_families=EMPTY_COLUMN_FAMILIES,
+            task_id="id"
+        )
+
+        mock_hook.return_value.get_column_families_for_table.return_value = \
+            EMPTY_COLUMN_FAMILIES
         instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance)
         mock_hook.return_value.create_table.side_effect = mock.Mock(
             side_effect=google.api_core.exceptions.AlreadyExists("Table already exists."))
@@ -409,7 +546,8 @@ def test_creating_table_that_exists(self, mock_hook):
             instance, TABLE_ID, INITIAL_SPLIT_KEYS, EMPTY_COLUMN_FAMILIES)
 
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_creating_table_that_exists_with_different_column_families_ids_in_the_table(self, mock_hook):
+    def test_creating_table_that_exists_with_different_column_families_ids_in_the_table(
+            self, mock_hook):
         op = BigtableTableCreateOperator(
             project_id=PROJECT_ID,
             instance_id=INSTANCE_ID,
@@ -419,7 +557,8 @@ def test_creating_table_that_exists_with_different_column_families_ids_in_the_ta
             task_id="id"
         )
 
-        mock_hook.return_value.get_column_families_for_table.return_value = {"existing_family": None}
+        mock_hook.return_value.get_column_families_for_table.return_value = {
+            "existing_family": None}
         mock_hook.return_value.create_table.side_effect = mock.Mock(
             side_effect=google.api_core.exceptions.AlreadyExists("Table already exists."))
 
@@ -433,7 +572,8 @@ def test_creating_table_that_exists_with_different_column_families_ids_in_the_ta
         mock_hook.assert_called_once_with()
 
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_creating_table_that_exists_with_different_column_families_gc_rule_in_the_table(self, mock_hook):
+    def test_creating_table_that_exists_with_different_column_families_gc_rule_in__table(
+            self, mock_hook):
         op = BigtableTableCreateOperator(
             project_id=PROJECT_ID,
             instance_id=INSTANCE_ID,
@@ -464,12 +604,12 @@ def test_creating_table_that_exists_with_different_column_families_gc_rule_in_th
 
 class BigtableWaitForTableReplicationTest(unittest.TestCase):
     @parameterized.expand([
-        ('project_id', '', INSTANCE_ID, TABLE_ID),
         ('instance_id', PROJECT_ID, '', TABLE_ID),
         ('table_id', PROJECT_ID, INSTANCE_ID, ''),
     ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0])
     @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook')
-    def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id, mock_hook):
+    def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id,
+                             mock_hook):
         with self.assertRaises(AirflowException) as e:
             BigtableTableWaitForReplicationSensor(
                 project_id=project_id,
diff --git a/tests/contrib/operators/test_gcp_bigtable_operator_system.py b/tests/contrib/operators/test_gcp_bigtable_operator_system.py
new file mode 100644
index 0000000000..525da7776c
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_bigtable_operator_system.py
@@ -0,0 +1,47 @@
+# -*- coding: utf-8 -*-
+#
+# 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 unittest
+
+from tests.contrib.operators.test_gcp_bigtable_operator_system_helper import \
+    GCPBigtableTestHelper
+from tests.contrib.utils.base_gcp_system_test_case import \
+    SKIP_TEST_WARNING, DagGcpSystemTestCase
+from tests.contrib.utils.gcp_authenticator import GCP_BIGTABLE_KEY
+
+
+@unittest.skipIf(DagGcpSystemTestCase.skip_check(GCP_BIGTABLE_KEY), SKIP_TEST_WARNING)
+class BigTableExampleDagsSystemTest(DagGcpSystemTestCase):
+    def __init__(self, method_name='runTest'):
+        super(BigTableExampleDagsSystemTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_bigtable_operators',
+            require_local_executor=True,
+            gcp_key=GCP_BIGTABLE_KEY)
+        self.helper = GCPBigtableTestHelper()
+
+    def test_run_example_dag_gcs_bigtable(self):
+        self._run_dag()
+
+    def tearDown(self):
+        self.gcp_authenticator.gcp_authenticate()
+        try:
+            self.helper.delete_instance()
+        finally:
+            self.gcp_authenticator.gcp_revoke_authentication()
+        super(BigTableExampleDagsSystemTest, self).tearDown()
diff --git a/tests/contrib/operators/test_gcp_bigtable_operator_system_helper.py b/tests/contrib/operators/test_gcp_bigtable_operator_system_helper.py
new file mode 100755
index 0000000000..4fe2828f03
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_bigtable_operator_system_helper.py
@@ -0,0 +1,67 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#
+# 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 os
+
+import argparse
+
+from tests.contrib.utils.gcp_authenticator import GcpAuthenticator, GCP_SPANNER_KEY
+from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor
+
+ITEST_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+ITEST_INSTANCE = os.environ.get('CBT_INSTANCE_ID', 'testinstance')
+
+
+class GCPBigtableTestHelper(LoggingCommandExecutor):
+
+    def delete_instance(self):
+        self.execute_cmd([
+            'gcloud', 'bigtable', '--project', ITEST_PROJECT_ID,
+            '--quiet', '--verbosity=none',
+            'instances', 'delete', ITEST_INSTANCE
+        ])
+
+
+if __name__ == '__main__':
+    parser = argparse.ArgumentParser(
+        description='Create or delete spanner instances for system tests.')
+    parser.add_argument('--action', dest='action', required=True,
+                        choices=('delete-instance',
+                                 'before-tests', 'after-tests'))
+    action = parser.parse_args().action
+
+    helper = GCPBigtableTestHelper()
+    gcp_authenticator = GcpAuthenticator(GCP_SPANNER_KEY)
+    helper.log.info('Starting action: {}'.format(action))
+
+    gcp_authenticator.gcp_store_authentication()
+    try:
+        gcp_authenticator.gcp_authenticate()
+        if action == 'before-tests':
+            pass
+        elif action == 'after-tests':
+            pass
+        elif action == 'delete-instance':
+            helper.delete_instance()
+        else:
+            raise Exception("Unknown action: {}".format(action))
+    finally:
+        gcp_authenticator.gcp_restore_authentication()
+
+    helper.log.info('Finishing action: {}'.format(action))
diff --git a/tests/contrib/operators/test_gcp_compute_operator.py b/tests/contrib/operators/test_gcp_compute_operator.py
index 4a4e336b7c..dbcafd2911 100644
--- a/tests/contrib/operators/test_gcp_compute_operator.py
+++ b/tests/contrib/operators/test_gcp_compute_operator.py
@@ -41,12 +41,12 @@
 
 EMPTY_CONTENT = ''.encode('utf8')
 
-PROJECT_ID = 'project-id'
-ZONE = 'zone'
+GCP_PROJECT_ID = 'project-id'
+GCE_ZONE = 'zone'
 RESOURCE_ID = 'resource-id'
-SHORT_MACHINE_TYPE_NAME = 'n1-machine-type'
+GCE_SHORT_MACHINE_TYPE_NAME = 'n1-machine-type'
 SET_MACHINE_TYPE_BODY = {
-    'machineType': 'zones/{}/machineTypes/{}'.format(ZONE, SHORT_MACHINE_TYPE_NAME)
+    'machineType': 'zones/{}/machineTypes/{}'.format(GCE_ZONE, GCE_SHORT_MACHINE_TYPE_NAME)
 }
 
 DEFAULT_DATE = timezone.datetime(2017, 1, 1)
@@ -57,8 +57,8 @@ class GceInstanceStartTest(unittest.TestCase):
     def test_instance_start(self, mock_hook):
         mock_hook.return_value.start_instance.return_value = True
         op = GceInstanceStartOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=RESOURCE_ID,
             task_id='id'
         )
@@ -66,14 +66,14 @@ def test_instance_start(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='v1',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.start_instance.assert_called_once_with(
-            PROJECT_ID, ZONE, RESOURCE_ID
+            GCP_PROJECT_ID, GCE_ZONE, RESOURCE_ID
         )
         self.assertTrue(result)
 
     # Setting all of the operator's input parameters as template dag_ids
     # (could be anything else) just to test if the templating works for all fields
     @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
-    def test_instance_start_with_templates(self, mock_hook):
+    def test_instance_start_with_templates(self, _):
         dag_id = 'test_dag_id'
         configuration.load_test_config()
         args = {
@@ -102,7 +102,7 @@ def test_start_should_throw_ex_when_missing_project_id(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceStartOperator(
                 project_id="",
-                zone=ZONE,
+                zone=GCE_ZONE,
                 resource_id=RESOURCE_ID,
                 task_id='id'
             )
@@ -111,11 +111,20 @@ def test_start_should_throw_ex_when_missing_project_id(self, mock_hook):
         self.assertIn("The required parameter 'project_id' is missing", str(err))
         mock_hook.assert_not_called()
 
+    @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
+    def test_start_should_not_throw_ex_when_project_id_None(self, _):
+        op = GceInstanceStartOperator(
+            zone=GCE_ZONE,
+            resource_id=RESOURCE_ID,
+            task_id='id'
+        )
+        op.execute(None)
+
     @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
     def test_start_should_throw_ex_when_missing_zone(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceStartOperator(
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 zone="",
                 resource_id=RESOURCE_ID,
                 task_id='id'
@@ -129,8 +138,8 @@ def test_start_should_throw_ex_when_missing_zone(self, mock_hook):
     def test_start_should_throw_ex_when_missing_resource_id(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceStartOperator(
-                project_id=PROJECT_ID,
-                zone=ZONE,
+                project_id=GCP_PROJECT_ID,
+                zone=GCE_ZONE,
                 resource_id="",
                 task_id='id'
             )
@@ -145,8 +154,8 @@ class GceInstanceStopTest(unittest.TestCase):
     def test_instance_stop(self, mock_hook):
         mock_hook.return_value.stop_instance.return_value = True
         op = GceInstanceStopOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=RESOURCE_ID,
             task_id='id'
         )
@@ -154,14 +163,14 @@ def test_instance_stop(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='v1',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.stop_instance.assert_called_once_with(
-            PROJECT_ID, ZONE, RESOURCE_ID
+            GCP_PROJECT_ID, GCE_ZONE, RESOURCE_ID
         )
         self.assertTrue(result)
 
     # Setting all of the operator's input parameters as templated dag_ids
     # (could be anything else) just to test if the templating works for all fields
     @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
-    def test_instance_stop_with_templates(self, mock_hook):
+    def test_instance_stop_with_templates(self, _):
         dag_id = 'test_dag_id'
         configuration.load_test_config()
         args = {
@@ -190,7 +199,7 @@ def test_stop_should_throw_ex_when_missing_project_id(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceStopOperator(
                 project_id="",
-                zone=ZONE,
+                zone=GCE_ZONE,
                 resource_id=RESOURCE_ID,
                 task_id='id'
             )
@@ -199,11 +208,20 @@ def test_stop_should_throw_ex_when_missing_project_id(self, mock_hook):
         self.assertIn("The required parameter 'project_id' is missing", str(err))
         mock_hook.assert_not_called()
 
+    @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
+    def test_stop_should_not_throw_ex_when_project_id_none(self, _):
+        op = GceInstanceStopOperator(
+            zone=GCE_ZONE,
+            resource_id=RESOURCE_ID,
+            task_id='id'
+        )
+        op.execute(None)
+
     @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
     def test_stop_should_throw_ex_when_missing_zone(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceStopOperator(
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 zone="",
                 resource_id=RESOURCE_ID,
                 task_id='id'
@@ -217,8 +235,8 @@ def test_stop_should_throw_ex_when_missing_zone(self, mock_hook):
     def test_stop_should_throw_ex_when_missing_resource_id(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceStopOperator(
-                project_id=PROJECT_ID,
-                zone=ZONE,
+                project_id=GCP_PROJECT_ID,
+                zone=GCE_ZONE,
                 resource_id="",
                 task_id='id'
             )
@@ -233,8 +251,8 @@ class GceInstanceSetMachineTypeTest(unittest.TestCase):
     def test_set_machine_type(self, mock_hook):
         mock_hook.return_value.set_machine_type.return_value = True
         op = GceSetMachineTypeOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=RESOURCE_ID,
             body=SET_MACHINE_TYPE_BODY,
             task_id='id'
@@ -243,14 +261,14 @@ def test_set_machine_type(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='v1',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.set_machine_type.assert_called_once_with(
-            PROJECT_ID, ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY
+            GCP_PROJECT_ID, GCE_ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY
         )
         self.assertTrue(result)
 
     # Setting all of the operator's input parameters as templated dag_ids
     # (could be anything else) just to test if the templating works for all fields
     @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
-    def test_set_machine_type_with_templates(self, mock_hook):
+    def test_set_machine_type_with_templates(self, _):
         dag_id = 'test_dag_id'
         configuration.load_test_config()
         args = {
@@ -280,7 +298,7 @@ def test_set_machine_type_should_throw_ex_when_missing_project_id(self, mock_hoo
         with self.assertRaises(AirflowException) as cm:
             op = GceSetMachineTypeOperator(
                 project_id="",
-                zone=ZONE,
+                zone=GCE_ZONE,
                 resource_id=RESOURCE_ID,
                 body=SET_MACHINE_TYPE_BODY,
                 task_id='id'
@@ -290,11 +308,21 @@ def test_set_machine_type_should_throw_ex_when_missing_project_id(self, mock_hoo
         self.assertIn("The required parameter 'project_id' is missing", str(err))
         mock_hook.assert_not_called()
 
+    @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
+    def test_set_machine_type_should_not_throw_ex_when_project_id_none(self, _):
+        op = GceSetMachineTypeOperator(
+            zone=GCE_ZONE,
+            resource_id=RESOURCE_ID,
+            body=SET_MACHINE_TYPE_BODY,
+            task_id='id'
+        )
+        op.execute(None)
+
     @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook')
     def test_set_machine_type_should_throw_ex_when_missing_zone(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceSetMachineTypeOperator(
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 zone="",
                 resource_id=RESOURCE_ID,
                 body=SET_MACHINE_TYPE_BODY,
@@ -309,8 +337,8 @@ def test_set_machine_type_should_throw_ex_when_missing_zone(self, mock_hook):
     def test_set_machine_type_should_throw_ex_when_missing_resource_id(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceSetMachineTypeOperator(
-                project_id=PROJECT_ID,
-                zone=ZONE,
+                project_id=GCP_PROJECT_ID,
+                zone=GCE_ZONE,
                 resource_id="",
                 body=SET_MACHINE_TYPE_BODY,
                 task_id='id'
@@ -324,8 +352,8 @@ def test_set_machine_type_should_throw_ex_when_missing_resource_id(self, mock_ho
     def test_set_machine_type_should_throw_ex_when_missing_machine_type(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             op = GceSetMachineTypeOperator(
-                project_id=PROJECT_ID,
-                zone=ZONE,
+                project_id=GCP_PROJECT_ID,
+                zone=GCE_ZONE,
                 resource_id=RESOURCE_ID,
                 body={},
                 task_id='id'
@@ -367,11 +395,12 @@ def test_set_machine_type_should_handle_and_trim_gce_error(
             self, get_conn, _execute_set_machine_type, _check_zone_operation_status):
         get_conn.return_value = {}
         _execute_set_machine_type.return_value = {"name": "test-operation"}
-        _check_zone_operation_status.return_value = ast.literal_eval(self.MOCK_OP_RESPONSE)
+        _check_zone_operation_status.return_value = ast.literal_eval(
+            self.MOCK_OP_RESPONSE)
         with self.assertRaises(AirflowException) as cm:
             op = GceSetMachineTypeOperator(
-                project_id=PROJECT_ID,
-                zone=ZONE,
+                project_id=GCP_PROJECT_ID,
+                zone=GCE_ZONE,
                 resource_id=RESOURCE_ID,
                 body=SET_MACHINE_TYPE_BODY,
                 task_id='id'
@@ -379,9 +408,9 @@ def test_set_machine_type_should_handle_and_trim_gce_error(
             op.execute(None)
         err = cm.exception
         _check_zone_operation_status.assert_called_once_with(
-            {}, "test-operation", PROJECT_ID, ZONE)
+            {}, "test-operation", GCP_PROJECT_ID, GCE_ZONE)
         _execute_set_machine_type.assert_called_once_with(
-            PROJECT_ID, ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY)
+            GCP_PROJECT_ID, GCE_ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY)
         # Checking the full message was sometimes failing due to different order
         # of keys in the serialized JSON
         self.assertIn("400 BAD REQUEST: {", str(err))  # checking the square bracket trim
@@ -490,7 +519,7 @@ def test_successful_copy_template(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             task_id='id',
             body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME}
@@ -499,7 +528,7 @@ def test_successful_copy_template(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='v1',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=GCE_INSTANCE_TEMPLATE_BODY_INSERT,
             request_id=None
         )
@@ -511,7 +540,7 @@ def test_idempotent_copy_template_when_already_copied(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             task_id='id',
             body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME}
@@ -530,7 +559,7 @@ def test_successful_copy_template_with_request_id(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID,
             task_id='id',
@@ -540,7 +569,7 @@ def test_successful_copy_template_with_request_id(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='v1',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=GCE_INSTANCE_TEMPLATE_BODY_INSERT,
             request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID,
         )
@@ -554,7 +583,7 @@ def test_successful_copy_template_with_description_fields(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID,
             task_id='id',
@@ -568,7 +597,7 @@ def test_successful_copy_template_with_description_fields(self, mock_hook):
         body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT)
         body_insert["description"] = "New description"
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=body_insert,
             request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID,
         )
@@ -582,7 +611,7 @@ def test_copy_with_some_validation_warnings(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             task_id='id',
             body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME,
@@ -598,7 +627,7 @@ def test_copy_with_some_validation_warnings(self, mock_hook):
         body_insert["some_wrong_field"] = "test"
         body_insert["properties"]["some_other_wrong_field"] = "test"
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=body_insert,
             request_id=None,
         )
@@ -612,7 +641,7 @@ def test_successful_copy_template_with_updated_nested_fields(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             task_id='id',
             body_patch={
@@ -628,7 +657,7 @@ def test_successful_copy_template_with_updated_nested_fields(self, mock_hook):
         body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT)
         body_insert["properties"]["machineType"] = "n1-standard-2"
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=body_insert,
             request_id=None
         )
@@ -642,7 +671,7 @@ def test_successful_copy_template_with_smaller_array_fields(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             task_id='id',
             body_patch={
@@ -681,7 +710,7 @@ def test_successful_copy_template_with_smaller_array_fields(self, mock_hook):
             }
         ]
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=body_insert,
             request_id=None
         )
@@ -695,7 +724,7 @@ def test_successful_copy_template_with_bigger_array_fields(self, mock_hook):
             GCE_INSTANCE_TEMPLATE_BODY_GET_NEW
         ]
         op = GceInstanceTemplateCopyOperator(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             resource_id=GCE_INSTANCE_TEMPLATE_NAME,
             task_id='id',
             body_patch={
@@ -742,7 +771,7 @@ def test_successful_copy_template_with_bigger_array_fields(self, mock_hook):
             }
         ]
         mock_hook.return_value.insert_instance_template.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             body=body_insert,
             request_id=None,
         )
@@ -757,7 +786,7 @@ def test_missing_name(self, mock_hook):
         ]
         with self.assertRaises(AirflowException) as cm:
             op = GceInstanceTemplateCopyOperator(
-                project_id=PROJECT_ID,
+                project_id=GCP_PROJECT_ID,
                 resource_id=GCE_INSTANCE_TEMPLATE_NAME,
                 request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID,
                 task_id='id',
@@ -876,8 +905,8 @@ def test_successful_instance_group_update(self, mock_hook):
         mock_hook.return_value.get_instance_group_manager.return_value = \
             deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET)
         op = GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             task_id='id',
             source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL,
@@ -887,8 +916,8 @@ def test_successful_instance_group_update(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='beta',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.patch_instance_group_manager.assert_called_once_with(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             body=GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH,
             request_id=None
@@ -902,8 +931,8 @@ def test_successful_instance_group_update_no_instance_template_field(self, mock_
         mock_hook.return_value.get_instance_group_manager.return_value = \
             instance_group_manager_no_template
         op = GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             task_id='id',
             source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL,
@@ -916,8 +945,8 @@ def test_successful_instance_group_update_no_instance_template_field(self, mock_
             deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH)
         del expected_patch_no_instance_template['instanceTemplate']
         mock_hook.return_value.patch_instance_group_manager.assert_called_once_with(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             body=expected_patch_no_instance_template,
             request_id=None
@@ -931,8 +960,8 @@ def test_successful_instance_group_update_no_versions_field(self, mock_hook):
         mock_hook.return_value.get_instance_group_manager.return_value = \
             instance_group_manager_no_versions
         op = GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             task_id='id',
             source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL,
@@ -945,8 +974,8 @@ def test_successful_instance_group_update_no_versions_field(self, mock_hook):
             deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH)
         del expected_patch_no_versions['versions']
         mock_hook.return_value.patch_instance_group_manager.assert_called_once_with(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             body=expected_patch_no_versions,
             request_id=None
@@ -958,8 +987,8 @@ def test_successful_instance_group_update_with_update_policy(self, mock_hook):
         mock_hook.return_value.get_instance_group_manager.return_value = \
             deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET)
         op = GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             task_id='id',
             update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY,
@@ -974,8 +1003,8 @@ def test_successful_instance_group_update_with_update_policy(self, mock_hook):
         expected_patch_with_update_policy['updatePolicy'] = \
             GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY
         mock_hook.return_value.patch_instance_group_manager.assert_called_once_with(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             body=expected_patch_with_update_policy,
             request_id=None
@@ -987,8 +1016,8 @@ def test_successful_instance_group_update_with_request_id(self, mock_hook):
         mock_hook.return_value.get_instance_group_manager.return_value = \
             deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET)
         op = GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             task_id='id',
             source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL,
@@ -999,8 +1028,8 @@ def test_successful_instance_group_update_with_request_id(self, mock_hook):
         mock_hook.assert_called_once_with(api_version='beta',
                                           gcp_conn_id='google_cloud_default')
         mock_hook.return_value.patch_instance_group_manager.assert_called_once_with(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             body=GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH,
             request_id=GCE_INSTANCE_GROUP_MANAGER_REQUEST_ID
@@ -1011,8 +1040,8 @@ def test_successful_instance_group_update_with_request_id(self, mock_hook):
     def test_try_to_use_api_v1(self, mock_hook):
         with self.assertRaises(AirflowException) as cm:
             GceInstanceGroupManagerUpdateTemplateOperator(
-                project_id=PROJECT_ID,
-                zone=ZONE,
+                project_id=GCP_PROJECT_ID,
+                zone=GCE_ZONE,
                 resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
                 task_id='id',
                 api_version='v1',
@@ -1027,8 +1056,8 @@ def test_try_to_use_non_existing_template(self, mock_hook):
         mock_hook.return_value.get_instance_group_manager.return_value = \
             deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET)
         op = GceInstanceGroupManagerUpdateTemplateOperator(
-            project_id=PROJECT_ID,
-            zone=ZONE,
+            project_id=GCP_PROJECT_ID,
+            zone=GCE_ZONE,
             resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME,
             task_id='id',
             source_template=GCE_INSTANCE_TEMPLATE_NON_EXISTING_URL,
diff --git a/tests/contrib/operators/test_gcp_compute_operator_system.py b/tests/contrib/operators/test_gcp_compute_operator_system.py
new file mode 100644
index 0000000000..0054df99d7
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_compute_operator_system.py
@@ -0,0 +1,84 @@
+# -*- coding: utf-8 -*-
+#
+# 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 unittest
+
+from tests.contrib.utils.base_gcp_system_test_case import \
+    SKIP_TEST_WARNING, DagGcpSystemTestCase
+
+from tests.contrib.operators.test_gcp_compute_operator_system_helper import \
+    GCPComputeTestHelper
+from tests.contrib.utils.gcp_authenticator import GCP_COMPUTE_KEY
+
+
+@unittest.skipIf(DagGcpSystemTestCase.skip_check(GCP_COMPUTE_KEY), SKIP_TEST_WARNING)
+class GcpComputeExampleDagsSystemTest(DagGcpSystemTestCase):
+
+    def setUp(self):
+        super(GcpComputeExampleDagsSystemTest, self).setUp()
+        self.gcp_authenticator.gcp_authenticate()
+        self.helper.delete_instance()
+        self.helper.create_instance()
+        self.gcp_authenticator.gcp_revoke_authentication()
+
+    def tearDown(self):
+        self.gcp_authenticator.gcp_authenticate()
+        self.helper.delete_instance()
+        self.gcp_authenticator.gcp_revoke_authentication()
+        super(GcpComputeExampleDagsSystemTest, self).tearDown()
+
+    def __init__(self, method_name='runTest'):
+        super(GcpComputeExampleDagsSystemTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_compute',
+            gcp_key=GCP_COMPUTE_KEY)
+        self.helper = GCPComputeTestHelper()
+
+    def test_run_example_dag_compute(self):
+        self._run_dag()
+
+
+@unittest.skipIf(DagGcpSystemTestCase.skip_check(GCP_COMPUTE_KEY), SKIP_TEST_WARNING)
+class GcpComputeIgmExampleDagsSystemTest(DagGcpSystemTestCase):
+
+    def setUp(self):
+        super(GcpComputeIgmExampleDagsSystemTest, self).setUp()
+        self.gcp_authenticator.gcp_authenticate()
+        try:
+            self.helper.delete_instance_group_and_template(silent=True)
+            self.helper.create_instance_group_and_template()
+        finally:
+            self.gcp_authenticator.gcp_revoke_authentication()
+
+    def tearDown(self):
+        self.gcp_authenticator.gcp_authenticate()
+        try:
+            self.helper.delete_instance_group_and_template()
+        finally:
+            self.gcp_authenticator.gcp_revoke_authentication()
+        super(GcpComputeIgmExampleDagsSystemTest, self).tearDown()
+
+    def __init__(self, method_name='runTest'):
+        super(GcpComputeIgmExampleDagsSystemTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_compute_igm',
+            gcp_key=GCP_COMPUTE_KEY)
+        self.helper = GCPComputeTestHelper()
+
+    def test_run_example_dag_compute_igm(self):
+        self._run_dag()
diff --git a/tests/contrib/operators/test_gcp_compute_operator_system_helper.py b/tests/contrib/operators/test_gcp_compute_operator_system_helper.py
new file mode 100755
index 0000000000..eaaffb9dd2
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_compute_operator_system_helper.py
@@ -0,0 +1,124 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#
+# 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 argparse
+import os
+
+from tests.contrib.utils.gcp_authenticator import GcpAuthenticator, GCP_COMPUTE_KEY
+from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor
+
+ITEST_INSTANCE = os.environ.get('GCE_INSTANCE', 'testinstance')
+ITEST_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+ITEST_INSTANCE_GROUP_MANAGER_NAME = os.environ.get('GCE_INSTANCE_GROUP_MANAGER_NAME',
+                                                   'instance-group-test')
+ITEST_ZONE = os.environ.get('GCE_ZONE', 'europe-west1-b')
+ITEST_TEMPLATE_NAME = os.environ.get('GCE_TEMPLATE_NAME',
+                                     'instance-template-test')
+ITEST_NEW_TEMPLATE_NAME = os.environ.get('GCE_NEW_TEMPLATE_NAME',
+                                         'instance-template-test-new')
+
+
+class GCPComputeTestHelper(LoggingCommandExecutor):
+
+    def delete_instance(self):
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID,
+            '--quiet', '--verbosity=none',
+            'instances', 'delete', ITEST_INSTANCE, '--zone', ITEST_ZONE,
+        ])
+
+    def create_instance(self):
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID, '--quiet',
+            'instances', 'create', ITEST_INSTANCE,
+            '--zone', ITEST_ZONE
+        ])
+
+    def delete_instance_group_and_template(self, silent=False):
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID,
+            '--quiet', '--verbosity=none',
+            'instance-groups', 'managed', 'delete', ITEST_INSTANCE_GROUP_MANAGER_NAME,
+            '--zone', ITEST_ZONE
+        ], silent=silent)
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID,
+            '--quiet', '--verbosity=none',
+            'instance-templates', 'delete', ITEST_NEW_TEMPLATE_NAME
+        ], silent=silent)
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute',
+            '--project', ITEST_PROJECT_ID,
+            '--quiet', '--verbosity=none',
+            'instance-templates', 'delete', ITEST_TEMPLATE_NAME
+        ], silent=silent)
+
+    def create_instance_group_and_template(self):
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID, '--quiet',
+            'instance-templates', 'create', ITEST_TEMPLATE_NAME
+        ])
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID, '--quiet',
+            'instance-groups', 'managed', 'create', ITEST_INSTANCE_GROUP_MANAGER_NAME,
+            '--template', ITEST_TEMPLATE_NAME,
+            '--zone', ITEST_ZONE, '--size=1'
+        ])
+        self.execute_cmd([
+            'gcloud', 'beta', 'compute', '--project', ITEST_PROJECT_ID, '--quiet',
+            'instance-groups', 'managed', 'wait-until-stable',
+            ITEST_INSTANCE_GROUP_MANAGER_NAME,
+            '--zone', ITEST_ZONE
+        ])
+
+
+if __name__ == '__main__':
+    parser = argparse.ArgumentParser(
+        description='Create or delete GCE instances/instance groups for system tests.')
+    parser.add_argument('--action', dest='action', required=True,
+                        choices=('create-instance', 'delete-instance',
+                                 'create-instance-group', 'delete-instance-group',
+                                 'before-tests', 'after-tests'))
+    action = parser.parse_args().action
+
+    helper = GCPComputeTestHelper()
+    gcp_authenticator = GcpAuthenticator(GCP_COMPUTE_KEY)
+    helper.log.info('Starting action: {}'.format(action))
+
+    gcp_authenticator.gcp_store_authentication()
+    try:
+        gcp_authenticator.gcp_authenticate()
+        if action == 'before-tests':
+            pass
+        elif action == 'after-tests':
+            pass
+        elif action == 'create-instance':
+            helper.create_instance()
+        elif action == 'delete-instance':
+            helper.delete_instance()
+        elif action == 'create-instance-group':
+            helper.create_instance_group_and_template()
+        elif action == 'delete-instance-group':
+            helper.delete_instance_group_and_template()
+        else:
+            raise Exception("Unknown action: {}".format(action))
+    finally:
+        gcp_authenticator.gcp_restore_authentication()
+
+    helper.log.info('Finishing action: {}'.format(action))
diff --git a/tests/contrib/operators/test_gcp_container_operator.py b/tests/contrib/operators/test_gcp_container_operator.py
index 1685e9c6ac..600c7cd8a8 100644
--- a/tests/contrib/operators/test_gcp_container_operator.py
+++ b/tests/contrib/operators/test_gcp_container_operator.py
@@ -33,7 +33,7 @@
     except ImportError:
         mock = None
 
-PROJECT_ID = 'test-id'
+TEST_GCP_PROJECT_ID = 'test-id'
 PROJECT_LOCATION = 'test-location'
 PROJECT_TASK_ID = 'test-task-id'
 CLUSTER_NAME = 'test-cluster-name'
@@ -55,7 +55,7 @@ class GoogleCloudPlatformContainerOperatorTest(unittest.TestCase):
 
     @mock.patch('airflow.contrib.operators.gcp_container_operator.GKEClusterHook')
     def test_create_execute(self, mock_hook):
-        operator = GKEClusterCreateOperator(project_id=PROJECT_ID,
+        operator = GKEClusterCreateOperator(project_id=TEST_GCP_PROJECT_ID,
                                             location=PROJECT_LOCATION,
                                             body=PROJECT_BODY_CREATE,
                                             task_id=PROJECT_TASK_ID)
@@ -67,7 +67,7 @@ def test_create_execute(self, mock_hook):
     @mock.patch('airflow.contrib.operators.gcp_container_operator.GKEClusterHook')
     def test_create_execute_error_body(self, mock_hook):
         with self.assertRaises(AirflowException):
-            operator = GKEClusterCreateOperator(project_id=PROJECT_ID,
+            operator = GKEClusterCreateOperator(project_id=TEST_GCP_PROJECT_ID,
                                                 location=PROJECT_LOCATION,
                                                 body=None,
                                                 task_id=PROJECT_TASK_ID)
@@ -88,7 +88,7 @@ def test_create_execute_error_project_id(self, mock_hook):
     @mock.patch('airflow.contrib.operators.gcp_container_operator.GKEClusterHook')
     def test_create_execute_error_location(self, mock_hook):
         with self.assertRaises(AirflowException):
-            operator = GKEClusterCreateOperator(project_id=PROJECT_ID,
+            operator = GKEClusterCreateOperator(project_id=TEST_GCP_PROJECT_ID,
                                                 body=PROJECT_BODY,
                                                 task_id=PROJECT_TASK_ID)
 
@@ -97,7 +97,7 @@ def test_create_execute_error_location(self, mock_hook):
 
     @mock.patch('airflow.contrib.operators.gcp_container_operator.GKEClusterHook')
     def test_delete_execute(self, mock_hook):
-        operator = GKEClusterDeleteOperator(project_id=PROJECT_ID,
+        operator = GKEClusterDeleteOperator(project_id=TEST_GCP_PROJECT_ID,
                                             name=CLUSTER_NAME,
                                             location=PROJECT_LOCATION,
                                             task_id=PROJECT_TASK_ID)
@@ -118,7 +118,7 @@ def test_delete_execute_error_project_id(self, mock_hook):
     @mock.patch('airflow.contrib.operators.gcp_container_operator.GKEClusterHook')
     def test_delete_execute_error_cluster_name(self, mock_hook):
         with self.assertRaises(AirflowException):
-            operator = GKEClusterDeleteOperator(project_id=PROJECT_ID,
+            operator = GKEClusterDeleteOperator(project_id=TEST_GCP_PROJECT_ID,
                                                 location=PROJECT_LOCATION,
                                                 task_id=PROJECT_TASK_ID)
 
@@ -128,7 +128,7 @@ def test_delete_execute_error_cluster_name(self, mock_hook):
     @mock.patch('airflow.contrib.operators.gcp_container_operator.GKEClusterHook')
     def test_delete_execute_error_location(self, mock_hook):
         with self.assertRaises(AirflowException):
-            operator = GKEClusterDeleteOperator(project_id=PROJECT_ID,
+            operator = GKEClusterDeleteOperator(project_id=TEST_GCP_PROJECT_ID,
                                                 name=CLUSTER_NAME,
                                                 task_id=PROJECT_TASK_ID)
 
@@ -138,7 +138,7 @@ def test_delete_execute_error_location(self, mock_hook):
 
 class GKEPodOperatorTest(unittest.TestCase):
     def setUp(self):
-        self.gke_op = GKEPodOperator(project_id=PROJECT_ID,
+        self.gke_op = GKEPodOperator(project_id=TEST_GCP_PROJECT_ID,
                                      location=PROJECT_LOCATION,
                                      cluster_name=CLUSTER_NAME,
                                      task_id=PROJECT_TASK_ID,
@@ -167,7 +167,7 @@ def test_execute_conn_id_none(self, proc_mock, file_mock, exec_mock):
 
         # Assert the gcloud command being called correctly
         proc_mock.assert_called_with(
-            GCLOUD_COMMAND.format(CLUSTER_NAME, PROJECT_LOCATION, PROJECT_ID).split())
+            GCLOUD_COMMAND.format(CLUSTER_NAME, PROJECT_LOCATION, TEST_GCP_PROJECT_ID).split())
 
         self.assertEqual(self.gke_op.config_file, FILE_NAME)
 
@@ -197,7 +197,7 @@ def test_execute_conn_id_path(self, proc_mock, file_mock, exec_mock, get_con_moc
 
         # Assert the gcloud command being called correctly
         proc_mock.assert_called_with(
-            GCLOUD_COMMAND.format(CLUSTER_NAME, PROJECT_LOCATION, PROJECT_ID).split())
+            GCLOUD_COMMAND.format(CLUSTER_NAME, PROJECT_LOCATION, TEST_GCP_PROJECT_ID).split())
 
         self.assertEqual(self.gke_op.config_file, FILE_NAME)
 
@@ -232,7 +232,7 @@ def test_execute_conn_id_dict(self, proc_mock, file_mock, exec_mock, get_con_moc
 
         # Assert the gcloud command being called correctly
         proc_mock.assert_called_with(
-            GCLOUD_COMMAND.format(CLUSTER_NAME, PROJECT_LOCATION, PROJECT_ID).split())
+            GCLOUD_COMMAND.format(CLUSTER_NAME, PROJECT_LOCATION, TEST_GCP_PROJECT_ID).split())
 
         self.assertEqual(self.gke_op.config_file, FILE_NAME)
 
diff --git a/tests/contrib/operators/test_gcp_function_operator.py b/tests/contrib/operators/test_gcp_function_operator.py
index 46d599bf7d..182434aec3 100644
--- a/tests/contrib/operators/test_gcp_function_operator.py
+++ b/tests/contrib/operators/test_gcp_function_operator.py
@@ -41,20 +41,21 @@
 EMPTY_CONTENT = ''.encode('utf8')
 MOCK_RESP_404 = type('', (object,), {"status": 404})()
 
-PROJECT_ID = 'test_project_id'
-LOCATION = 'test_region'
-SOURCE_ARCHIVE_URL = 'gs://folder/file.zip'
-ENTRYPOINT = 'helloWorld'
-FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(PROJECT_ID, LOCATION,
-                                                               ENTRYPOINT)
-RUNTIME = 'nodejs6'
+GCP_PROJECT_ID = 'test_project_id'
+GCP_LOCATION = 'test_region'
+GCF_SOURCE_ARCHIVE_URL = 'gs://folder/file.zip'
+GCF_ENTRYPOINT = 'helloWorld'
+FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(GCP_PROJECT_ID,
+                                                               GCP_LOCATION,
+                                                               GCF_ENTRYPOINT)
+GCF_RUNTIME = 'nodejs6'
 VALID_RUNTIMES = ['nodejs6', 'nodejs8', 'python37']
 VALID_BODY = {
     "name": FUNCTION_NAME,
-    "entryPoint": ENTRYPOINT,
-    "runtime": RUNTIME,
+    "entryPoint": GCF_ENTRYPOINT,
+    "runtime": GCF_RUNTIME,
     "httpsTrigger": {},
-    "sourceArchiveUrl": SOURCE_ARCHIVE_URL
+    "sourceArchiveUrl": GCF_SOURCE_ARCHIVE_URL
 }
 
 
@@ -100,8 +101,8 @@ def test_deploy_execute(self, mock_hook):
             side_effect=HttpError(resp=MOCK_RESP_404, content=b'not found'))
         mock_hook.return_value.create_new_function.return_value = True
         op = GcfFunctionDeployOperator(
-            project_id=PROJECT_ID,
-            location=LOCATION,
+            project_id=GCP_PROJECT_ID,
+            location=GCP_LOCATION,
             body=deepcopy(VALID_BODY),
             task_id="id"
         )
@@ -116,8 +117,7 @@ def test_deploy_execute(self, mock_hook):
             'airflow-version': 'v' + version.replace('.', '-').replace('+', '-')
         }
         mock_hook.return_value.create_new_function.assert_called_once_with(
-            'projects/test_project_id/locations/test_region',
-            expected_body
+            'test_project_id', 'test_region', expected_body
         )
 
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
@@ -125,8 +125,8 @@ def test_update_function_if_exists(self, mock_hook):
         mock_hook.return_value.get_function.return_value = True
         mock_hook.return_value.update_function.return_value = True
         op = GcfFunctionDeployOperator(
-            project_id=PROJECT_ID,
-            location=LOCATION,
+            project_id=GCP_PROJECT_ID,
+            location=GCP_LOCATION,
             body=deepcopy(VALID_BODY),
             task_id="id"
         )
@@ -146,18 +146,22 @@ def test_update_function_if_exists(self, mock_hook):
         mock_hook.return_value.create_new_function.assert_not_called()
 
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
-    def test_empty_project_id(self, mock_hook):
-        with self.assertRaises(AirflowException) as cm:
-            GcfFunctionDeployOperator(
-                project_id="",
-                location="test_region",
-                body=None,
-                task_id="id"
-            )
-        err = cm.exception
-        self.assertIn("The required parameter 'project_id' is missing", str(err))
+    def test_empty_project_id_is_ok(self, mock_hook):
+        operator = GcfFunctionDeployOperator(
+            location="test_region",
+            body=deepcopy(VALID_BODY),
+            task_id="id"
+        )
+        operator._hook.get_function.side_effect = \
+            HttpError(resp=MOCK_RESP_404, content=b'not found')
+        operator.execute(None)
         mock_hook.assert_called_once_with(api_version='v1',
                                           gcp_conn_id='google_cloud_default')
+        new_body = deepcopy(VALID_BODY)
+        new_body['labels'] = {
+            'airflow-version': 'v' + version.replace('.', '-').replace('+', '-')}
+        mock_hook.return_value.create_new_function.assert_called_once_with(
+            None, "test_region", new_body)
 
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_empty_location(self, mock_hook):
@@ -192,7 +196,6 @@ def test_empty_body(self, mock_hook):
     ])
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_correct_runtime_field(self, runtime, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = deepcopy(VALID_BODY)
         body['runtime'] = runtime
@@ -217,7 +220,6 @@ def test_correct_runtime_field(self, runtime, mock_hook):
     ])
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_valid_network_field(self, network, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = deepcopy(VALID_BODY)
         body['network'] = network
@@ -241,7 +243,6 @@ def test_valid_network_field(self, network, mock_hook):
     ])
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_valid_labels_field(self, labels, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = deepcopy(VALID_BODY)
         body['labels'] = labels
@@ -258,7 +259,6 @@ def test_valid_labels_field(self, labels, mock_hook):
 
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_validation_disabled(self, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = {
             "name": "function_name",
@@ -278,7 +278,6 @@ def test_validation_disabled(self, mock_hook):
 
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_body_validation_simple(self, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = deepcopy(VALID_BODY)
         body['name'] = ''
@@ -315,7 +314,6 @@ def test_body_validation_simple(self, mock_hook):
     ])
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_invalid_field_values(self, key, value, message, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = deepcopy(VALID_BODY)
         body[key] = value
@@ -424,14 +422,15 @@ def test_valid_source_code_union_field(self, source_code, mock_hook):
                                           gcp_conn_id='google_cloud_default')
         if zip_path:
             mock_hook.return_value.upload_function_zip.assert_called_once_with(
-                parent='projects/test_project_id/locations/test_region',
+                project_id='test_project_id',
+                location='test_region',
                 zip_path='/path/to/file'
             )
         mock_hook.return_value.get_function.assert_called_once_with(
             'projects/test_project_id/locations/test_region/functions/helloWorld'
         )
         mock_hook.return_value.create_new_function.assert_called_once_with(
-            'projects/test_project_id/locations/test_region',
+            'test_project_id', 'test_region',
             body
         )
         mock_hook.reset_mock()
@@ -514,14 +513,12 @@ def test_valid_trigger_union_field(self, trigger, mock_hook):
             'projects/test_project_id/locations/test_region/functions/helloWorld'
         )
         mock_hook.return_value.create_new_function.assert_called_once_with(
-            'projects/test_project_id/locations/test_region',
-            body
+            'test_project_id', 'test_region', body
         )
         mock_hook.reset_mock()
 
     @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook')
     def test_extra_parameter(self, mock_hook):
-        mock_hook.return_value.list_functions.return_value = []
         mock_hook.return_value.create_new_function.return_value = True
         body = deepcopy(VALID_BODY)
         body['extra_parameter'] = 'extra'
diff --git a/tests/contrib/operators/test_gcp_function_operator_system.py b/tests/contrib/operators/test_gcp_function_operator_system.py
new file mode 100644
index 0000000000..2dc24de00a
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_function_operator_system.py
@@ -0,0 +1,37 @@
+# -*- coding: utf-8 -*-
+#
+# 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 unittest
+
+from tests.contrib.utils.base_gcp_system_test_case import \
+    SKIP_TEST_WARNING, DagGcpSystemTestCase
+from tests.contrib.utils.gcp_authenticator import GCP_FUNCTION_KEY
+
+
+@unittest.skipIf(
+    DagGcpSystemTestCase.skip_check(GCP_FUNCTION_KEY), SKIP_TEST_WARNING)
+class GcpFunctionExampleDagsSystemTest(DagGcpSystemTestCase):
+    def __init__(self, method_name='runTest'):
+        super(GcpFunctionExampleDagsSystemTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_function',
+            gcp_key=GCP_FUNCTION_KEY)
+
+    def test_run_example_dag_function(self):
+        self._run_dag()
diff --git a/tests/contrib/operators/test_gcp_spanner_operator.py b/tests/contrib/operators/test_gcp_spanner_operator.py
index 525f796471..71a2005cb6 100644
--- a/tests/contrib/operators/test_gcp_spanner_operator.py
+++ b/tests/contrib/operators/test_gcp_spanner_operator.py
@@ -22,11 +22,12 @@
 
 from airflow import AirflowException
 from airflow.contrib.operators.gcp_spanner_operator import \
-    CloudSpannerInstanceDeployOperator, CloudSpannerInstanceDeleteOperator, \
-    CloudSpannerInstanceDatabaseQueryOperator, CloudSpannerInstanceDatabaseDeployOperator, \
-    CloudSpannerInstanceDatabaseDeleteOperator, CloudSpannerInstanceDatabaseUpdateOperator
-from tests.contrib.operators.test_gcp_base import BaseGcpIntegrationTestCase, \
-    SKIP_TEST_WARNING, GCP_SPANNER_KEY
+    CloudSpannerInstanceDeployOperator, \
+    CloudSpannerInstanceDeleteOperator, \
+    CloudSpannerInstanceDatabaseQueryOperator, \
+    CloudSpannerInstanceDatabaseDeployOperator, \
+    CloudSpannerInstanceDatabaseDeleteOperator, \
+    CloudSpannerInstanceDatabaseUpdateOperator
 
 try:
     # noinspection PyProtectedMember
@@ -45,8 +46,8 @@
 DISPLAY_NAME = 'Test Instance'
 INSERT_QUERY = "INSERT my_table1 (id, name) VALUES (1, 'One')"
 INSERT_QUERY_2 = "INSERT my_table2 (id, name) VALUES (1, 'One')"
-CREATE_QUERY = "CREATE TABLE my_table1 (id INT64, name STRING(MAX)) PRIMARY KEY (id)"
-CREATE_QUERY_2 = "CREATE TABLE my_table2 (id INT64, name STRING(MAX)) PRIMARY KEY (id)"
+CREATE_QUERY = "CREATE TABLE my_table1 (id INT64, name STRING(100))"
+CREATE_QUERY_2 = "CREATE TABLE my_table2 (id INT64, name STRING(100))"
 DDL_STATEMENTS = [CREATE_QUERY, CREATE_QUERY_2]
 
 
@@ -70,6 +71,24 @@ def test_instance_create(self, mock_hook):
         mock_hook.return_value.update_instance.assert_not_called()
         self.assertTrue(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_instance_create_missing_project_id(self, mock_hook):
+        mock_hook.return_value.get_instance.return_value = None
+        op = CloudSpannerInstanceDeployOperator(
+            instance_id=INSTANCE_ID,
+            configuration_name=CONFIG_NAME,
+            node_count=int(NODE_COUNT),
+            display_name=DISPLAY_NAME,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.create_instance.assert_called_once_with(
+            None, INSTANCE_ID, CONFIG_NAME, int(NODE_COUNT), DISPLAY_NAME
+        )
+        mock_hook.return_value.update_instance.assert_not_called()
+        self.assertTrue(result)
+
     @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
     def test_instance_update(self, mock_hook):
         mock_hook.return_value.get_instance.return_value = {"name": INSTANCE_ID}
@@ -89,6 +108,24 @@ def test_instance_update(self, mock_hook):
         mock_hook.return_value.create_instance.assert_not_called()
         self.assertTrue(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_instance_update_missing_project_id(self, mock_hook):
+        mock_hook.return_value.get_instance.return_value = {"name": INSTANCE_ID}
+        op = CloudSpannerInstanceDeployOperator(
+            instance_id=INSTANCE_ID,
+            configuration_name=CONFIG_NAME,
+            node_count=int(NODE_COUNT),
+            display_name=DISPLAY_NAME,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.update_instance.assert_called_once_with(
+            None, INSTANCE_ID, CONFIG_NAME, int(NODE_COUNT), DISPLAY_NAME
+        )
+        mock_hook.return_value.create_instance.assert_not_called()
+        self.assertTrue(result)
+
     @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
     def test_instance_create_aborts_and_succeeds_if_instance_exists(self, mock_hook):
         mock_hook.return_value.get_instance.return_value = {"name": INSTANCE_ID}
@@ -140,6 +177,20 @@ def test_instance_delete(self, mock_hook):
         )
         self.assertTrue(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_instance_delete_missing_project_id(self, mock_hook):
+        mock_hook.return_value.get_instance.return_value = {"name": INSTANCE_ID}
+        op = CloudSpannerInstanceDeleteOperator(
+            instance_id=INSTANCE_ID,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.delete_instance.assert_called_once_with(
+            None, INSTANCE_ID
+        )
+        self.assertTrue(result)
+
     @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
     def test_instance_delete_aborts_and_succeeds_if_instance_does_not_exist(self,
                                                                             mock_hook):
@@ -188,6 +239,22 @@ def test_instance_query(self, mock_hook):
         )
         self.assertIsNone(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_instance_query_missing_project_id(self, mock_hook):
+        mock_hook.return_value.execute_sql.return_value = None
+        op = CloudSpannerInstanceDatabaseQueryOperator(
+            instance_id=INSTANCE_ID,
+            database_id=DB_ID,
+            query=INSERT_QUERY,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.execute_dml.assert_called_once_with(
+            None, INSTANCE_ID, DB_ID, [INSERT_QUERY]
+        )
+        self.assertIsNone(result)
+
     @parameterized.expand([
         ("", INSTANCE_ID, DB_ID, INSERT_QUERY, "project_id"),
         (PROJECT_ID, "", DB_ID, INSERT_QUERY, "instance_id"),
@@ -260,6 +327,24 @@ def test_database_create(self, mock_hook):
         mock_hook.return_value.update_database.assert_not_called()
         self.assertTrue(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_database_create_missing_project_id(self, mock_hook):
+        mock_hook.return_value.get_database.return_value = None
+        op = CloudSpannerInstanceDatabaseDeployOperator(
+            instance_id=INSTANCE_ID,
+            database_id=DB_ID,
+            ddl_statements=DDL_STATEMENTS,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.create_database.assert_called_once_with(
+            project_id=None, instance_id=INSTANCE_ID, database_id=DB_ID,
+            ddl_statements=DDL_STATEMENTS
+        )
+        mock_hook.return_value.update_database.assert_not_called()
+        self.assertTrue(result)
+
     @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
     def test_database_create_with_pre_existing_db(self, mock_hook):
         mock_hook.return_value.get_database.return_value = {"name": DB_ID}
@@ -316,6 +401,23 @@ def test_database_update(self, mock_hook):
         )
         self.assertTrue(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_database_update_missing_project_id(self, mock_hook):
+        mock_hook.return_value.get_database.return_value = {"name": DB_ID}
+        op = CloudSpannerInstanceDatabaseUpdateOperator(
+            instance_id=INSTANCE_ID,
+            database_id=DB_ID,
+            ddl_statements=DDL_STATEMENTS,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.update_database.assert_called_once_with(
+            project_id=None, instance_id=INSTANCE_ID, database_id=DB_ID,
+            ddl_statements=DDL_STATEMENTS, operation_id=None
+        )
+        self.assertTrue(result)
+
     @parameterized.expand([
         ("", INSTANCE_ID, DB_ID, DDL_STATEMENTS, 'project_id'),
         (PROJECT_ID, "", DB_ID, DDL_STATEMENTS, 'instance_id'),
@@ -370,6 +472,21 @@ def test_database_delete(self, mock_hook):
         )
         self.assertTrue(result)
 
+    @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
+    def test_database_delete_missing_project_id(self, mock_hook):
+        mock_hook.return_value.get_database.return_value = {"name": DB_ID}
+        op = CloudSpannerInstanceDatabaseDeleteOperator(
+            instance_id=INSTANCE_ID,
+            database_id=DB_ID,
+            task_id="id"
+        )
+        result = op.execute(None)
+        mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default")
+        mock_hook.return_value.delete_database.assert_called_once_with(
+            project_id=None, instance_id=INSTANCE_ID, database_id=DB_ID
+        )
+        self.assertTrue(result)
+
     @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook")
     def test_database_delete_exits_and_succeeds_if_database_does_not_exist(self,
                                                                            mock_hook):
@@ -405,16 +522,3 @@ def test_database_delete_ex_if_param_missing(self, project_id, instance_id,
         err = cm.exception
         self.assertIn("The required parameter '{}' is empty".format(exp_msg), str(err))
         mock_hook.assert_not_called()
-
-
-@unittest.skipIf(
-    BaseGcpIntegrationTestCase.skip_check(GCP_SPANNER_KEY), SKIP_TEST_WARNING)
-class CloudSpannerExampleDagsTest(BaseGcpIntegrationTestCase):
-    def __init__(self, method_name='runTest'):
-        super(CloudSpannerExampleDagsTest, self).__init__(
-            method_name,
-            dag_id='example_gcp_spanner',
-            gcp_key=GCP_SPANNER_KEY)
-
-    def test_run_example_dag_cloudsql_query(self):
-        self._run_dag()
diff --git a/tests/contrib/operators/test_gcp_spanner_operator_system.py b/tests/contrib/operators/test_gcp_spanner_operator_system.py
new file mode 100644
index 0000000000..fd4c50bd8a
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_spanner_operator_system.py
@@ -0,0 +1,46 @@
+# -*- coding: utf-8 -*-
+#
+# 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 unittest
+
+from tests.contrib.operators.test_gcp_spanner_operator_system_helper import \
+    GCPSpannerTestHelper
+from tests.contrib.utils.base_gcp_system_test_case import \
+    SKIP_TEST_WARNING, DagGcpSystemTestCase
+from tests.contrib.utils.gcp_authenticator import GCP_SPANNER_KEY
+
+
+@unittest.skipIf(DagGcpSystemTestCase.skip_check(GCP_SPANNER_KEY), SKIP_TEST_WARNING)
+class CloudSpannerExampleDagsTest(DagGcpSystemTestCase):
+    def __init__(self, method_name='runTest'):
+        super(CloudSpannerExampleDagsTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_spanner',
+            gcp_key=GCP_SPANNER_KEY)
+        self.helper = GCPSpannerTestHelper()
+
+    def tearDown(self):
+        self.gcp_authenticator.gcp_authenticate()
+        try:
+            self.helper.delete_instance()
+        finally:
+            self.gcp_authenticator.gcp_revoke_authentication()
+        super(CloudSpannerExampleDagsTest, self).tearDown()
+
+    def test_run_example_dag_spanner(self):
+        self._run_dag()
diff --git a/tests/contrib/operators/test_gcp_spanner_operator_system_helper.py b/tests/contrib/operators/test_gcp_spanner_operator_system_helper.py
new file mode 100755
index 0000000000..36af11f5e4
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_spanner_operator_system_helper.py
@@ -0,0 +1,67 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#
+# 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 os
+
+import argparse
+
+from tests.contrib.utils.gcp_authenticator import GcpAuthenticator, GCP_SPANNER_KEY
+from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor
+
+ITEST_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+ITEST_INSTANCE = os.environ.get('GCP_SPANNER_INSTANCE_ID', 'testinstance')
+
+
+class GCPSpannerTestHelper(LoggingCommandExecutor):
+
+    def delete_instance(self):
+        self.execute_cmd([
+            'gcloud', 'spanner', '--project', ITEST_PROJECT_ID,
+            '--quiet', '--verbosity=none',
+            'instances', 'delete', ITEST_INSTANCE
+        ])
+
+
+if __name__ == '__main__':
+    parser = argparse.ArgumentParser(
+        description='Create or delete spanner instances for system tests.')
+    parser.add_argument('--action', dest='action', required=True,
+                        choices=('delete-instance',
+                                 'before-tests', 'after-tests'))
+    action = parser.parse_args().action
+
+    helper = GCPSpannerTestHelper()
+    gcp_authenticator = GcpAuthenticator(GCP_SPANNER_KEY)
+    helper.log.info('Starting action: {}'.format(action))
+
+    gcp_authenticator.gcp_store_authentication()
+    try:
+        gcp_authenticator.gcp_authenticate()
+        if action == 'before-tests':
+            pass
+        elif action == 'after-tests':
+            pass
+        elif action == 'delete-instance':
+            helper.delete_instance()
+        else:
+            raise Exception("Unknown action: {}".format(action))
+    finally:
+        gcp_authenticator.gcp_restore_authentication()
+
+    helper.log.info('Finishing action: {}'.format(action))
diff --git a/tests/contrib/operators/test_gcp_sql_operator.py b/tests/contrib/operators/test_gcp_sql_operator.py
index 646b2e771f..e3d6d7a651 100644
--- a/tests/contrib/operators/test_gcp_sql_operator.py
+++ b/tests/contrib/operators/test_gcp_sql_operator.py
@@ -20,20 +20,21 @@
 import os
 import unittest
 
-import time
 from parameterized import parameterized
-from uuid import uuid1
 
 from airflow import AirflowException
-from airflow.contrib.hooks.gcp_sql_hook import CloudSqlProxyRunner
-from airflow.contrib.operators.gcp_sql_operator import CloudSqlInstanceCreateOperator, \
-    CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator, \
-    CloudSqlInstanceDatabaseCreateOperator, CloudSqlInstanceDatabasePatchOperator, \
-    CloudSqlInstanceExportOperator, CloudSqlInstanceImportOperator, \
-    CloudSqlInstanceDatabaseDeleteOperator, CloudSqlQueryOperator
+from airflow.contrib.operators.gcp_sql_operator import \
+    CloudSqlInstanceCreateOperator, \
+    CloudSqlInstancePatchOperator, \
+    CloudSqlInstanceDeleteOperator, \
+    CloudSqlInstanceDatabaseCreateOperator, \
+    CloudSqlInstanceDatabasePatchOperator, \
+    CloudSqlInstanceExportOperator, \
+    CloudSqlInstanceImportOperator, \
+    CloudSqlInstanceDatabaseDeleteOperator, \
+    CloudSqlQueryOperator
 from airflow.models.connection import Connection
-from tests.contrib.operators.test_gcp_base import BaseGcpIntegrationTestCase, \
-    GCP_CLOUDSQL_KEY, SKIP_TEST_WARNING
+from tests.contrib.utils.gcp_authenticator import PROJECT_EXTRA
 
 try:
     # noinspection PyProtectedMember
@@ -150,7 +151,7 @@
             "schemaOnly": False
         },
         "csvExportOptions": {
-            "selectQuery": "SELECT * FROM ..."
+            "selectQuery": "SELECT * FROM TABLE"
         }
     }
 }
@@ -546,6 +547,16 @@ def test_instance_import(self, mock_hook):
 
 
 class CloudSqlQueryValidationTest(unittest.TestCase):
+
+    @staticmethod
+    def _setup_connections(get_connections, uri):
+        gcp_connection = mock.MagicMock()
+        gcp_connection.extra_dejson = mock.MagicMock()
+        gcp_connection.extra_dejson.get.return_value = {PROJECT_EXTRA: 'example_project'}
+        cloudsql_connection = Connection()
+        cloudsql_connection.parse_from_uri(uri)
+        get_connections.side_effect = [[gcp_connection], [cloudsql_connection]]
+
     @parameterized.expand([
         ('', 'location', 'instance_name', 'postgres', False, False,
          'SELECT * FROM TEST',
@@ -602,12 +613,10 @@ def test_create_operator_with_wrong_parameters(self,
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_postgres(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=False&use_ssl=False")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=False&use_ssl=False"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -625,13 +634,11 @@ def test_create_operator_with_correct_parameters_postgres(self, get_connections)
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_postgres_ssl(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=False&use_ssl=True&sslcert=/bin/bash&"
-            "sslkey=/bin/bash&sslrootcert=/bin/bash")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=False&use_ssl=True&sslcert=/bin/bash&" \
+              "sslkey=/bin/bash&sslrootcert=/bin/bash"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -653,12 +660,10 @@ def test_create_operator_with_correct_parameters_postgres_ssl(self, get_connecti
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_postgres_proxy_socket(
             self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=True&sql_proxy_use_tcp=False")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=True&sql_proxy_use_tcp=False"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -678,12 +683,10 @@ def test_create_operator_with_correct_parameters_postgres_proxy_socket(
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_postgres_proxy_tcp(self,
                                                                         get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=True&sql_proxy_use_tcp=True")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=True&sql_proxy_use_tcp=True"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -701,12 +704,32 @@ def test_create_operator_with_correct_parameters_postgres_proxy_tcp(self,
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_mysql(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=False&use_ssl=False")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&" \
+            "project_id=example-project&location=europe-west1&instance=testdb&" \
+            "use_proxy=False&use_ssl=False"
+        self._setup_connections(get_connections, uri)
+        operator = CloudSqlQueryOperator(
+            sql=['SELECT * FROM TABLE'],
+            task_id='task_id'
+        )
+        operator.cloudsql_db_hook.create_connection()
+        try:
+            db_hook = operator.cloudsql_db_hook.get_database_hook()
+            conn = db_hook._get_connections_from_db(db_hook.mysql_conn_id)[0]
+        finally:
+            operator.cloudsql_db_hook.delete_connection()
+        self.assertEqual('mysql', conn.conn_type)
+        self.assertEqual('8.8.8.8', conn.host)
+        self.assertEqual(3200, conn.port)
+        self.assertEqual('testdb', conn.schema)
+
+    @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
+    def test_create_operator_with_correct_parameters_project_id_missing(self,
+                                                                        get_connections):
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&" \
+              "location=europe-west1&instance=testdb&" \
+              "use_proxy=False&use_ssl=False"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -724,13 +747,11 @@ def test_create_operator_with_correct_parameters_mysql(self, get_connections):
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_mysql_ssl(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=False&use_ssl=True&sslcert=/bin/bash&"
-            "sslkey=/bin/bash&sslrootcert=/bin/bash")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=False&use_ssl=True&sslcert=/bin/bash&" \
+              "sslkey=/bin/bash&sslrootcert=/bin/bash"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -752,12 +773,10 @@ def test_create_operator_with_correct_parameters_mysql_ssl(self, get_connections
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_mysql_proxy_socket(self,
                                                                         get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=True&sql_proxy_use_tcp=False")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=True&sql_proxy_use_tcp=False"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -778,12 +797,10 @@ def test_create_operator_with_correct_parameters_mysql_proxy_socket(self,
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_correct_parameters_mysql_tcp(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&"
-            "project_id=example-project&location=europe-west1&instance=testdb&"
-            "use_proxy=True&sql_proxy_use_tcp=True")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=mysql&" \
+              "project_id=example-project&location=europe-west1&instance=testdb&" \
+              "use_proxy=True&sql_proxy_use_tcp=True"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -801,33 +818,30 @@ def test_create_operator_with_correct_parameters_mysql_tcp(self, get_connections
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_too_long_unix_socket_path(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&"
-            "project_id=example-project&location=europe-west1&"
-            "instance="
-            "test_db_with_long_name_a_bit_above_the_limit_of_UNIX_socket&"
-            "use_proxy=True&sql_proxy_use_tcp=False")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" \
+              "project_id=example-project&location=europe-west1&" \
+              "instance=" \
+              "test_db_with_long_name_a_bit_above" \
+              "_the_limit_of_UNIX_socket_asdadadasadasd&" \
+              "use_proxy=True&sql_proxy_use_tcp=False"
+        self._setup_connections(get_connections, uri)
+        operator = CloudSqlQueryOperator(
+            sql=['SELECT * FROM TABLE'],
+            task_id='task_id'
+        )
         with self.assertRaises(AirflowException) as cm:
-            op = CloudSqlQueryOperator(
-                sql=['SELECT * FROM TABLE'],
-                task_id='task_id'
-            )
-            op.execute(None)
+            operator.execute(None)
         err = cm.exception
         self.assertIn("The UNIX socket path length cannot exceed", str(err))
 
     @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections")
     def test_create_operator_with_not_too_long_unix_socket_path(self, get_connections):
-        connection = Connection()
-        connection.parse_from_uri(
-            "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&"
-            "project_id=example-project&location=europe-west1&"
-            "instance="
-            "test_db_with_longname_but_with_limit_of_UNIX_socket_aaaa&"
-            "use_proxy=True&sql_proxy_use_tcp=False")
-        get_connections.return_value = [connection]
+        uri = "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" \
+              "project_id=example-project&location=europe-west1&" \
+              "instance=" \
+              "test_db_with_longname_but_with_limit_of_UNIX_socket&" \
+              "use_proxy=True&sql_proxy_use_tcp=False"
+        self._setup_connections(get_connections, uri)
         operator = CloudSqlQueryOperator(
             sql=['SELECT * FROM TABLE'],
             task_id='task_id'
@@ -873,89 +887,3 @@ def test_cloudsql_hook_delete_connection_on_exception(
         err = cm.exception
         self.assertEqual("Exception when running a query", str(err))
         delete_connection.assert_called_once_with()
-
-
-@unittest.skipIf(
-    BaseGcpIntegrationTestCase.skip_check(GCP_CLOUDSQL_KEY), SKIP_TEST_WARNING)
-class CloudSqlProxyIntegrationTest(BaseGcpIntegrationTestCase):
-    def __init__(self, method_name='runTest'):
-        super(CloudSqlProxyIntegrationTest, self).__init__(
-            method_name,
-            dag_id='example_gcp_sql_query',
-            gcp_key='gcp_cloudsql.json')
-
-    def test_start_proxy_fail_no_parameters(self):
-        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + str(uuid1()),
-                                     project_id=PROJECT_ID,
-                                     instance_specification='a')
-        with self.assertRaises(AirflowException) as cm:
-            runner.start_proxy()
-        err = cm.exception
-        self.assertIn("invalid instance name", str(err))
-        with self.assertRaises(AirflowException) as cm:
-            runner.start_proxy()
-        err = cm.exception
-        self.assertIn("invalid instance name", str(err))
-        self.assertIsNone(runner.sql_proxy_process)
-
-    def test_start_proxy_with_all_instances(self):
-        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + str(uuid1()),
-                                     project_id=PROJECT_ID,
-                                     instance_specification='')
-        try:
-            runner.start_proxy()
-            time.sleep(1)
-        finally:
-            runner.stop_proxy()
-        self.assertIsNone(runner.sql_proxy_process)
-
-    def test_start_proxy_with_all_instances_generated_credential_file(self):
-        self.update_connection_with_dictionary()
-        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + str(uuid1()),
-                                     project_id=PROJECT_ID,
-                                     instance_specification='')
-        try:
-            runner.start_proxy()
-            time.sleep(1)
-        finally:
-            runner.stop_proxy()
-        self.assertIsNone(runner.sql_proxy_process)
-
-    def test_start_proxy_with_all_instances_specific_version(self):
-        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + str(uuid1()),
-                                     project_id=PROJECT_ID,
-                                     instance_specification='',
-                                     sql_proxy_version='v1.13')
-        try:
-            runner.start_proxy()
-            time.sleep(1)
-        finally:
-            runner.stop_proxy()
-        self.assertIsNone(runner.sql_proxy_process)
-        self.assertEqual(runner.get_proxy_version(), "1.13")
-
-
-@unittest.skipIf(
-    BaseGcpIntegrationTestCase.skip_check(GCP_CLOUDSQL_KEY), SKIP_TEST_WARNING)
-class CloudSqlExampleDagsIntegrationTest(BaseGcpIntegrationTestCase):
-    def __init__(self, method_name='runTest'):
-        super(CloudSqlExampleDagsIntegrationTest, self).__init__(
-            method_name,
-            dag_id='example_gcp_sql',
-            gcp_key=GCP_CLOUDSQL_KEY)
-
-    def test_run_example_dag_cloudsql_query(self):
-        self._run_dag()
-
-
-@unittest.skipIf(
-    BaseGcpIntegrationTestCase.skip_check(GCP_CLOUDSQL_KEY), SKIP_TEST_WARNING)
-class CloudSqlQueryExampleDagsIntegrationTest(BaseGcpIntegrationTestCase):
-    def __init__(self, method_name='runTest'):
-        super(CloudSqlQueryExampleDagsIntegrationTest, self).__init__(
-            method_name,
-            dag_id='example_gcp_sql_query',
-            gcp_key=GCP_CLOUDSQL_KEY)
-
-    def test_run_example_dag_cloudsql_query(self):
-        self._run_dag()
diff --git a/tests/contrib/operators/test_gcp_sql_operator_system.py b/tests/contrib/operators/test_gcp_sql_operator_system.py
new file mode 100644
index 0000000000..4fc7b56136
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_sql_operator_system.py
@@ -0,0 +1,64 @@
+# -*- coding: utf-8 -*-
+#
+# 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 os
+import unittest
+
+from airflow import AirflowException
+from tests.contrib.utils.base_gcp_system_test_case import \
+    SKIP_TEST_WARNING, DagGcpSystemTestCase
+from tests.contrib.operators.test_gcp_sql_operator_system_helper import \
+    CloudSqlQueryTestHelper
+from tests.contrib.utils.gcp_authenticator import GCP_CLOUDSQL_KEY
+
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'project-id')
+
+SQL_QUERY_TEST_HELPER = CloudSqlQueryTestHelper()
+
+
+@unittest.skipIf(DagGcpSystemTestCase.skip_check(GCP_CLOUDSQL_KEY), SKIP_TEST_WARNING)
+class CloudSqlExampleDagsIntegrationTest(DagGcpSystemTestCase):
+    def __init__(self, method_name='runTest'):
+        super(CloudSqlExampleDagsIntegrationTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_sql',
+            gcp_key=GCP_CLOUDSQL_KEY)
+
+    def tearDown(self):
+        # Delete instances just in case the test failed and did not cleanup after itself
+        self.gcp_authenticator.gcp_authenticate()
+        try:
+            SQL_QUERY_TEST_HELPER.delete_instances()
+            SQL_QUERY_TEST_HELPER.delete_instances(instance_suffix="2")
+            SQL_QUERY_TEST_HELPER.delete_service_account_acls()
+        finally:
+            self.gcp_authenticator.gcp_revoke_authentication()
+        super(CloudSqlExampleDagsIntegrationTest, self).tearDown()
+
+    def test_run_example_dag_cloudsql(self):
+        try:
+            self._run_dag()
+        except AirflowException as e:
+            self.log.warning(
+                "In case you see 'The instance or operation is not in an appropriate "
+                "state to handle the request' error - you "
+                "can remove '.random' file from airflow folder and re-run "
+                "the test. This will generate random name of the database for next run "
+                "(the problem is that Cloud SQL keeps names of deleted instances in "
+                "short-term cache).")
+            raise e
diff --git a/tests/contrib/operators/test_gcp_sql_operator_system_helper.py b/tests/contrib/operators/test_gcp_sql_operator_system_helper.py
new file mode 100755
index 0000000000..02f85da4b7
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_sql_operator_system_helper.py
@@ -0,0 +1,478 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#
+# 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 errno
+import json
+import os
+from os.path import expanduser
+
+import argparse
+from threading import Thread
+
+import time
+
+# This is just to retrieve variables from Airflow Breeze in case it is available
+from six.moves.urllib.parse import urlsplit
+
+from tests.contrib.utils.base_gcp_system_test_case import RetrieveVariables
+from tests.contrib.utils.gcp_authenticator import GcpAuthenticator, GCP_CLOUDSQL_KEY
+from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor
+
+try:
+    # noinspection PyProtectedMember
+    from unittest import mock
+except ImportError:
+    try:
+        import mock
+    except ImportError:
+        mock = None
+
+retrieve_variables = RetrieveVariables()
+
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project')
+GCP_LOCATION = os.environ.get('GCP_LOCATION', 'europe-west1')
+
+GCSQL_POSTGRES_SERVER_CA_FILE = os.environ.get('GCSQL_POSTGRES_SERVER_CA_FILE',
+                                               ".key/postgres-server-ca.pem")
+GCSQL_POSTGRES_CLIENT_CERT_FILE = os.environ.get('GCSQL_POSTGRES_CLIENT_CERT_FILE',
+                                                 ".key/postgres-client-cert.pem")
+GCSQL_POSTGRES_CLIENT_KEY_FILE = os.environ.get('GCSQL_POSTGRES_CLIENT_KEY_FILE',
+                                                ".key/postgres-client-key.pem")
+GCSQL_POSTGRES_PUBLIC_IP_FILE = os.environ.get('GCSQL_POSTGRES_PUBLIC_IP_FILE',
+                                               ".key/postgres-ip.env")
+GCSQL_POSTGRES_USER = os.environ.get('GCSQL_POSTGRES_USER', 'postgres_user')
+GCSQL_POSTGRES_DATABASE_NAME = os.environ.get('GCSQL_POSTGRES_DATABASE_NAME',
+                                              'postgresdb')
+GCSQL_MYSQL_CLIENT_CERT_FILE = os.environ.get('GCSQL_MYSQL_CLIENT_CERT_FILE',
+                                              ".key/mysql-client-cert.pem")
+GCSQL_MYSQL_CLIENT_KEY_FILE = os.environ.get('GCSQL_MYSQL_CLIENT_KEY_FILE',
+                                             ".key/mysql-client-key.pem")
+GCSQL_MYSQL_SERVER_CA_FILE = os.environ.get('GCSQL_MYSQL_SERVER_CA_FILE',
+                                            ".key/mysql-server-ca.pem")
+GCSQL_MYSQL_PUBLIC_IP_FILE = os.environ.get('GCSQL_MYSQL_PUBLIC_IP_FILE',
+                                            ".key/mysql-ip.env")
+GCSQL_MYSQL_USER = os.environ.get('GCSQL_MYSQL_USER', 'mysql_user')
+GCSQL_MYSQL_DATABASE_NAME = os.environ.get('GCSQL_MYSQL_DATABASE_NAME', 'mysqldb')
+
+GCSQL_MYSQL_EXPORT_URI = os.environ.get('GCSQL_MYSQL_EXPORT_URI',
+                                        'gs://bucketName/fileName')
+DB_VERSION_MYSQL = 'MYSQL_5_7'
+DV_VERSION_POSTGRES = 'POSTGRES_9_6'
+
+HOME_DIR = expanduser("~")
+
+
+def get_absolute_path(path):
+    if path.startswith("/"):
+        return path
+    else:
+        return os.path.join(HOME_DIR, path)
+
+
+server_ca_file_postgres = get_absolute_path(GCSQL_POSTGRES_SERVER_CA_FILE)
+client_cert_file_postgres = get_absolute_path(GCSQL_POSTGRES_CLIENT_CERT_FILE)
+client_key_file_postgres = get_absolute_path(GCSQL_POSTGRES_CLIENT_KEY_FILE)
+
+server_ca_file_mysql = get_absolute_path(GCSQL_MYSQL_SERVER_CA_FILE)
+client_cert_file_mysql = get_absolute_path(GCSQL_MYSQL_CLIENT_CERT_FILE)
+client_key_file_mysql = get_absolute_path(GCSQL_MYSQL_CLIENT_KEY_FILE)
+
+
+def get_postgres_instance_name(instance_suffix=''):
+    return os.environ.get('GCSQL_POSTGRES_INSTANCE_NAME' + instance_suffix,
+                          'testpostgres')
+
+
+def get_mysql_instance_name(instance_suffix=''):
+    return os.environ.get('GCSQL_MYSQL_INSTANCE_NAME' + instance_suffix,
+                          'testmysql')
+
+
+class CloudSqlQueryTestHelper(LoggingCommandExecutor):
+
+    def create_instances(self, instance_suffix=''):
+        thread_mysql = Thread(target=lambda: self.__create_instance(
+            get_mysql_instance_name(instance_suffix), DB_VERSION_MYSQL))
+        thread_postgres = Thread(target=lambda: self.__create_instance(
+            get_postgres_instance_name(instance_suffix), DV_VERSION_POSTGRES))
+        thread_mysql.start()
+        thread_postgres.start()
+        thread_mysql.join()
+        thread_postgres.join()
+
+    def delete_instances(self, instance_suffix=''):
+        thread_mysql = Thread(target=lambda: self.__delete_instance(
+            get_mysql_instance_name(instance_suffix)))
+        thread_postgres = Thread(target=lambda: self.__delete_instance(
+            get_postgres_instance_name(instance_suffix)))
+        thread_mysql.start()
+        thread_postgres.start()
+        thread_mysql.join()
+        thread_postgres.join()
+
+    def get_ip_addresses(self, instance_suffix):
+        with open(GCSQL_MYSQL_PUBLIC_IP_FILE, "w") as f:
+            ip = self.__get_ip_address(get_mysql_instance_name(instance_suffix),
+                                       'GCSQL_MYSQL_PUBLIC_IP')
+            f.write(ip)
+        with open(GCSQL_POSTGRES_PUBLIC_IP_FILE, "w") as f:
+            ip = self.__get_ip_address(get_postgres_instance_name(instance_suffix),
+                                       'GCSQL_POSTGRES_PUBLIC_IP')
+            f.write(ip)
+
+    def raise_database_exception(self, database):
+        raise Exception("The {database} instance does not exist. Make sure to run  "
+                        "`python {f} --action=before-tests` before running the test"
+                        " (and remember to run `python {f} --action=after-tests` "
+                        "after you are done."
+                        .format(f=__file__, database=database))
+
+    def check_if_instances_are_up(self, instance_suffix=''):
+        res_postgres = self.execute_cmd(
+            ['gcloud', 'sql', 'instances', 'describe',
+             get_postgres_instance_name(instance_suffix),
+             "--project={}".format(GCP_PROJECT_ID)])
+        if res_postgres != 0:
+            self.raise_database_exception('postgres')
+        res_postgres = self.execute_cmd(
+            ['gcloud', 'sql', 'instances', 'describe',
+             get_postgres_instance_name(instance_suffix),
+             "--project={}".format(GCP_PROJECT_ID)])
+        if res_postgres != 0:
+            self.raise_database_exception('mysql')
+
+    def authorize_address(self, instance_suffix=''):
+        ip = self.__get_my_public_ip()
+        self.log.info('Authorizing access from IP: %s', ip)
+        postgres_thread = Thread(target=lambda: self.execute_cmd(
+            ['gcloud', 'sql', 'instances', 'patch',
+             get_postgres_instance_name(instance_suffix), '--quiet',
+             "--authorized-networks={}".format(ip),
+             "--project={}".format(GCP_PROJECT_ID)]))
+        mysql_thread = Thread(target=lambda: self.execute_cmd(
+            ['gcloud', 'sql', 'instances', 'patch',
+             get_mysql_instance_name(instance_suffix), '--quiet',
+             "--authorized-networks={}".format(ip),
+             "--project={}".format(GCP_PROJECT_ID)]))
+        postgres_thread.start()
+        mysql_thread.start()
+        postgres_thread.join()
+        mysql_thread.join()
+
+    def setup_instances(self, instance_suffix=''):
+        mysql_thread = Thread(target=lambda: self.__setup_instance_and_certs(
+            get_mysql_instance_name(instance_suffix), DB_VERSION_MYSQL,
+            server_ca_file_mysql,
+            client_key_file_mysql, client_cert_file_mysql, GCSQL_MYSQL_DATABASE_NAME,
+            GCSQL_MYSQL_USER
+        ))
+        postgres_thread = Thread(target=lambda: self.__setup_instance_and_certs(
+            get_postgres_instance_name(instance_suffix), DV_VERSION_POSTGRES,
+            server_ca_file_postgres,
+            client_key_file_postgres, client_cert_file_postgres,
+            GCSQL_POSTGRES_DATABASE_NAME, GCSQL_POSTGRES_USER
+        ))
+        mysql_thread.start()
+        postgres_thread.start()
+        mysql_thread.join()
+        postgres_thread.join()
+        self.get_ip_addresses(instance_suffix)
+        self.authorize_address(instance_suffix)
+
+    def delete_service_account_acls(self):
+        self.__delete_service_accounts_acls()
+
+    def __create_instance(self, instance_name, db_version):
+        self.log.info('Creating a test %s instance "%s"...', db_version, instance_name)
+        try:
+            create_instance_opcode = self.__create_sql_instance(instance_name, db_version)
+            if create_instance_opcode:  # return code 1, some error occurred
+                operation_name = self.__get_operation_name(instance_name)
+                self.log.info('Waiting for operation: %s ...', operation_name)
+                self.__wait_for_create(operation_name)
+                self.log.info('... Done.')
+
+            self.log.info('... Done creating a test %s instance "%s"!\n',
+                          db_version, instance_name)
+        except Exception as ex:
+            self.log.error('Exception occurred. '
+                           'Aborting creating a test instance.\n\n%s', ex)
+            raise ex
+
+    def __delete_service_accounts_acls(self):
+        export_bucket_split = urlsplit(GCSQL_MYSQL_EXPORT_URI)
+        export_bucket_name = export_bucket_split[1]  # netloc (bucket)
+        self.log.info('Deleting temporary service accounts from bucket "%s"...',
+                      export_bucket_name)
+        all_permissions = self.check_output(['gsutil', 'iam', 'get',
+                                             "gs://{}".format(export_bucket_name),
+                                             "--project={}".format(GCP_PROJECT_ID)])
+        all_permissions_dejson = json.loads(all_permissions.decode("utf-8"))
+        for binding in all_permissions_dejson['bindings']:
+            if binding['role'] == 'roles/storage.legacyBucketWriter':
+                for member in binding['members']:
+                    if not member.startswith('serviceAccount:gcp-storage-account'):
+
+                        self.log.info("Remove member: {}".format(member))
+                        member_type, member_email = member.split(':')
+                        if member_type != 'serviceAccount':
+                            self.log.warning("Skip removing member {} as the type {} is "
+                                             "not service account".format(member,
+                                                                          member_type))
+                        self.execute_cmd(['gsutil', 'acl', 'ch', '-d', member_email,
+                                         "gs://{}".format(export_bucket_name)])
+                    else:
+                        self.log.info("Skip removing member {}".format(member))
+
+    @staticmethod
+    def set_ip_addresses_in_env():
+        CloudSqlQueryTestHelper.__set_ip_address_in_env(GCSQL_MYSQL_PUBLIC_IP_FILE)
+        CloudSqlQueryTestHelper.__set_ip_address_in_env(GCSQL_POSTGRES_PUBLIC_IP_FILE)
+
+    @staticmethod
+    def __set_ip_address_in_env(file_name):
+        if os.path.exists(file_name):
+            with open(file_name, "r") as f:
+                env, ip = f.read().split("=")
+                os.environ[env] = ip
+
+    def __setup_instance_and_certs(self, instance_name, db_version, server_ca_file,
+                                   client_key_file, client_cert_file, db_name,
+                                   db_username):
+        self.log.info('Setting up a test %s instance "%s"...', db_version, instance_name)
+        try:
+            self.__remove_keys_and_certs([server_ca_file, client_key_file,
+                                          client_cert_file])
+
+            self.__wait_for_operations(instance_name)
+            self.__write_to_file(server_ca_file, self.__get_server_ca_cert(instance_name))
+            client_cert_name = 'client-cert-name'
+            self.__wait_for_operations(instance_name)
+            self.__delete_client_cert(instance_name, client_cert_name)
+            self.__wait_for_operations(instance_name)
+            self.__create_client_cert(instance_name, client_key_file, client_cert_name)
+            self.__wait_for_operations(instance_name)
+            self.__write_to_file(client_cert_file,
+                                 self.__get_client_cert(instance_name, client_cert_name))
+            self.__wait_for_operations(instance_name)
+            self.__wait_for_operations(instance_name)
+            self.__create_user(instance_name, db_username)
+            self.__wait_for_operations(instance_name)
+            self.__delete_db(instance_name, db_name)
+            self.__create_db(instance_name, db_name)
+            self.log.info('... Done setting up a test %s instance "%s"!\n',
+                          db_version, instance_name)
+        except Exception as ex:
+            self.log.error('Exception occurred. '
+                           'Aborting setting up test instance and certs.\n\n%s', ex)
+            raise ex
+
+    def __delete_instance(self, instance_name):
+        # type: (str) -> None
+        self.log.info('Deleting Cloud SQL instance "%s"...', instance_name)
+        self.execute_cmd(['gcloud', 'sql', 'instances', 'delete',
+                          instance_name, '--quiet'])
+        self.log.info('... Done.')
+
+    def __get_my_public_ip(self):
+        return self.check_output(
+            ['curl', 'https://ipinfo.io/ip']).decode('utf-8').strip()
+
+    def __create_sql_instance(self, instance_name, db_version):
+        # type: (str, str) -> int
+        return self.execute_cmd(
+            ['gcloud', 'sql', 'instances', 'create', instance_name,
+             '--region', GCP_LOCATION,
+             '--project', GCP_PROJECT_ID,
+             '--database-version', db_version,
+             '--tier', 'db-f1-micro'])
+
+    def __get_server_ca_cert(self, instance_name):
+        # type: (str) -> bytes
+        self.log.info('Getting server CA cert for "%s"...', instance_name)
+        output = self.check_output(
+            ['gcloud', 'sql', 'instances', 'describe', instance_name,
+             '--format=value(serverCaCert.cert)'])
+        self.log.info('... Done.')
+        return output
+
+    def __get_client_cert(self, instance_name, client_cert_name):
+        # type: (str, str) -> bytes
+        self.log.info('Getting client cert for "%s"...', instance_name)
+        output = self.check_output(
+            ['gcloud', 'sql', 'ssl', 'client-certs', 'describe', client_cert_name, '-i',
+             instance_name, '--format=get(cert)'])
+        self.log.info('... Done.')
+        return output
+
+    def __create_user(self, instance_name, username):
+        # type: (str, str) -> None
+        self.log.info('Creating user "%s" in Cloud SQL instance "%s"...', username,
+                      instance_name)
+        self.execute_cmd(['gcloud', 'sql', 'users', 'create', username, '-i',
+                          instance_name, '--host', '%', '--password', 'JoxHlwrPzwch0gz9',
+                          '--quiet'])
+        self.log.info('... Done.')
+
+    def __delete_db(self, instance_name, db_name):
+        # type: (str, str) -> None
+        self.log.info('Deleting database "%s" in Cloud SQL instance "%s"...', db_name,
+                      instance_name)
+        self.execute_cmd(['gcloud', 'sql', 'databases', 'delete', db_name, '-i',
+                          instance_name, '--quiet'])
+        self.log.info('... Done.')
+
+    def __create_db(self, instance_name, db_name):
+        # type: (str, str) -> None
+        self.log.info('Creating database "%s" in Cloud SQL instance "%s"...', db_name,
+                      instance_name)
+        self.execute_cmd(['gcloud', 'sql', 'databases', 'create', db_name, '-i',
+                          instance_name, '--quiet'])
+        self.log.info('... Done.')
+
+    def __write_to_file(self, filepath, content):
+        # type: (str, bytes) -> None
+        # https://stackoverflow.com/a/12517490
+        self.log.info("Checking file under: %s", filepath)
+        if not os.path.exists(os.path.dirname(filepath)):
+            self.log.info("File doesn't exits. Creating dir...")
+            try:
+                os.makedirs(os.path.dirname(filepath))
+            except OSError as exc:  # Guard against race condition
+                self.log.info("Error while creating dir.")
+                if exc.errno != errno.EEXIST:
+                    raise
+        self.log.info("... Done. Dir created.")
+
+        with open(filepath, "w") as f:
+            f.write(str(content.decode('utf-8')))
+        self.log.info('Written file in: %s', filepath)
+
+    def __remove_keys_and_certs(self, filepaths):
+        if not len(filepaths):
+            return
+        self.log.info('Removing client keys and certs...')
+
+        for filepath in filepaths:
+            if os.path.exists(filepath):
+                os.remove(filepath)
+        self.log.info('Done ...')
+
+    def __delete_client_cert(self, instance_name, common_name):
+        self.log.info('Deleting client key and cert for "%s"...', instance_name)
+        self.execute_cmd(['gcloud', 'sql', 'ssl', 'client-certs', 'delete', common_name,
+                          '-i', instance_name, '--quiet'])
+        self.log.info('... Done.')
+
+    def __create_client_cert(self, instance_name, client_key_file, common_name):
+        self.log.info('Creating client key and cert for "%s"...', instance_name)
+        try:
+            os.remove(client_key_file)
+        except OSError:
+            pass
+        self.execute_cmd(['gcloud', 'sql', 'ssl', 'client-certs', 'create', common_name,
+                          client_key_file, '-i', instance_name])
+        self.log.info('... Done.')
+
+    def __get_operation_name(self, instance_name):
+        # type: (str) -> str
+        op_name_bytes = self.check_output(
+            ['gcloud', 'sql', 'operations', 'list', '-i',
+             instance_name, '--format=get(name)'])
+        return op_name_bytes.decode('utf-8').strip()
+
+    def __print_operations(self, operations):
+        self.log.info("\n==== OPERATIONS >>>>")
+        self.log.info(operations)
+        self.log.info("<<<< OPERATIONS ====\n")
+
+    def __wait_for_operations(self, instance_name):
+        # type: (str) -> None
+        while True:
+            operations = self.__get_operations(instance_name)
+            self.__print_operations(operations)
+            if "RUNNING" in operations:
+                self.log.info("Found a running operation. Sleeping 5s before retrying...")
+                time.sleep(5)
+            else:
+                break
+
+    def __get_ip_address(self, instance_name, env_var):
+        # type: (str, str) -> str
+        ip = self.check_output(
+            ['gcloud', 'sql', 'instances', 'describe',
+             instance_name,
+             '--format=get(ipAddresses[0].ipAddress)']
+        ).decode('utf-8').strip()
+        os.environ[env_var] = ip
+        return "{}={}".format(env_var, ip)
+
+    def __get_operations(self, instance_name):
+        # type: (str) -> str
+        op_name_bytes = self.check_output(
+            ['gcloud', 'sql', 'operations', 'list', '-i',
+             instance_name, '--format=get(NAME,TYPE,STATUS)'])
+        return op_name_bytes.decode('utf-8').strip()
+
+    def __wait_for_create(self, operation_name):
+        # type: (str) -> None
+        self.execute_cmd(['gcloud', 'beta', 'sql', 'operations', 'wait',
+                          '--project', GCP_PROJECT_ID, operation_name])
+
+
+if __name__ == '__main__':
+    parser = argparse.ArgumentParser(
+        description='Create or delete Cloud SQL instances for system tests.')
+    parser.add_argument('--action', required=True,
+                        choices=('create', 'delete', 'setup-instances',
+                                 'create2', 'delete2', 'setup-instances2',
+                                 'before-tests', 'after-tests',
+                                 'delete-service-accounts-acls'))
+    action = parser.parse_args().action
+
+    helper = CloudSqlQueryTestHelper()
+    gcp_authenticator = GcpAuthenticator(GCP_CLOUDSQL_KEY)
+    helper.log.info('Starting action: {}'.format(action))
+
+    gcp_authenticator.gcp_store_authentication()
+    try:
+        gcp_authenticator.gcp_authenticate()
+        if action == 'before-tests':
+            pass
+        elif action == 'after-tests':
+            pass
+        elif action == 'create':
+            helper.create_instances()
+        elif action == 'delete':
+            helper.delete_instances()
+        elif action == 'create2':
+            helper.create_instances(instance_suffix="2")
+        elif action == 'delete2':
+            helper.delete_instances(instance_suffix="2")
+        elif action == 'setup-instances':
+            helper.setup_instances()
+        elif action == 'setup-instances2':
+            helper.setup_instances(instance_suffix="2")
+        elif action == 'delete-service-accounts-acls':
+            helper.delete_service_account_acls()
+        else:
+            raise Exception("Unknown action: {}".format(action))
+    finally:
+        gcp_authenticator.gcp_restore_authentication()
+    helper.log.info('Finishing action: {}'.format(action))
diff --git a/tests/contrib/operators/test_gcp_sql_operatorquery_system.py b/tests/contrib/operators/test_gcp_sql_operatorquery_system.py
new file mode 100644
index 0000000000..4e216662f6
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_sql_operatorquery_system.py
@@ -0,0 +1,146 @@
+# -*- coding: utf-8 -*-
+#
+# 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 os
+import random
+import string
+import unittest
+from os.path import dirname
+
+import time
+
+from airflow import AirflowException
+from airflow.contrib.hooks.gcp_sql_hook import CloudSqlProxyRunner
+from tests.contrib.utils.base_gcp_system_test_case import BaseGcpSystemTestCase, \
+    DagGcpSystemTestCase
+from tests.contrib.operators.test_gcp_sql_operator_system_helper import \
+    CloudSqlQueryTestHelper
+from tests.contrib.utils.gcp_authenticator import GCP_CLOUDSQL_KEY
+
+GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'project-id')
+
+SKIP_CLOUDSQL_QUERY_WARNING = """
+    This test is skipped from automated runs intentionally
+    as creating databases in Google Cloud SQL takes a very
+    long time. You can still set GCP_ENABLE_CLOUDSQL_QUERY_TEST
+    environment variable to 'True' and then you should be able to
+    run it manually after you create the database
+    Creating the database can be done by running
+    `{}/test_gcp_sql_operatorquery_system_helper.py \
+--action=before-tests`
+    (you should remember to delete the database with --action=after-tests flag)
+""".format(dirname(__file__))
+
+GCP_ENABLE_CLOUDSQL_QUERY_TEST = os.environ.get('GCP_ENABLE_CLOUDSQL_QUERY_TEST')
+
+if GCP_ENABLE_CLOUDSQL_QUERY_TEST == 'True':
+    enable_cloudsql_query_test = True
+else:
+    enable_cloudsql_query_test = False
+
+
+SQL_QUERY_TEST_HELPER = CloudSqlQueryTestHelper()
+
+
+@unittest.skipIf(not enable_cloudsql_query_test, SKIP_CLOUDSQL_QUERY_WARNING)
+class CloudSqlProxySystemTest(BaseGcpSystemTestCase):
+    def __init__(self, method_name='runTest'):
+        super(CloudSqlProxySystemTest, self).__init__(
+            method_name,
+            gcp_key='gcp_cloudsql.json')
+
+    def setUp(self):
+        super(CloudSqlProxySystemTest, self).setUp()
+        self.gcp_authenticator.gcp_authenticate()
+        SQL_QUERY_TEST_HELPER.check_if_instances_are_up(instance_suffix="_QUERY")
+        self.gcp_authenticator.gcp_revoke_authentication()
+
+    @staticmethod
+    def generate_unique_path():
+        return ''.join(
+            random.choice(string.ascii_letters + string.digits) for _ in range(8))
+
+    def test_start_proxy_fail_no_parameters(self):
+        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + self.generate_unique_path(),
+                                     project_id=GCP_PROJECT_ID,
+                                     instance_specification='a')
+        with self.assertRaises(AirflowException) as cm:
+            runner.start_proxy()
+        err = cm.exception
+        self.assertIn("invalid instance name", str(err))
+        with self.assertRaises(AirflowException) as cm:
+            runner.start_proxy()
+        err = cm.exception
+        self.assertIn("invalid instance name", str(err))
+        self.assertIsNone(runner.sql_proxy_process)
+
+    def test_start_proxy_with_all_instances(self):
+        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + self.generate_unique_path(),
+                                     project_id=GCP_PROJECT_ID,
+                                     instance_specification='')
+        try:
+            runner.start_proxy()
+            time.sleep(1)
+        finally:
+            runner.stop_proxy()
+        self.assertIsNone(runner.sql_proxy_process)
+
+    def test_start_proxy_with_all_instances_generated_credential_file(self):
+        self.gcp_authenticator.set_dictionary_in_airflow_connection()
+        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + self.generate_unique_path(),
+                                     project_id=GCP_PROJECT_ID,
+                                     instance_specification='')
+        try:
+            runner.start_proxy()
+            time.sleep(1)
+        finally:
+            runner.stop_proxy()
+        self.assertIsNone(runner.sql_proxy_process)
+
+    def test_start_proxy_with_all_instances_specific_version(self):
+        runner = CloudSqlProxyRunner(path_prefix='/tmp/' + self.generate_unique_path(),
+                                     project_id=GCP_PROJECT_ID,
+                                     instance_specification='',
+                                     sql_proxy_version='v1.13')
+        try:
+            runner.start_proxy()
+            time.sleep(1)
+        finally:
+            runner.stop_proxy()
+        self.assertIsNone(runner.sql_proxy_process)
+        self.assertEqual(runner.get_proxy_version(), "1.13")
+
+
+@unittest.skipIf(not enable_cloudsql_query_test, SKIP_CLOUDSQL_QUERY_WARNING)
+class CloudSqlQueryExampleDagsSystemTest(DagGcpSystemTestCase):
+
+    def __init__(self, method_name='runTest'):
+        super(CloudSqlQueryExampleDagsSystemTest, self).__init__(
+            method_name,
+            dag_id='example_gcp_sql_query',
+            gcp_key=GCP_CLOUDSQL_KEY)
+
+    def setUp(self):
+        super(CloudSqlQueryExampleDagsSystemTest, self).setUp()
+        self.gcp_authenticator.gcp_authenticate()
+        SQL_QUERY_TEST_HELPER.check_if_instances_are_up(instance_suffix="_QUERY")
+        SQL_QUERY_TEST_HELPER.setup_instances(instance_suffix="_QUERY")
+        self.gcp_authenticator.gcp_revoke_authentication()
+
+    def test_run_example_dag_cloudsql_query(self):
+        self._run_dag()
diff --git a/tests/contrib/operators/test_gcp_sql_operatorquery_system_helper.py b/tests/contrib/operators/test_gcp_sql_operatorquery_system_helper.py
new file mode 100755
index 0000000000..22a643a130
--- /dev/null
+++ b/tests/contrib/operators/test_gcp_sql_operatorquery_system_helper.py
@@ -0,0 +1,60 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#
+# 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 argparse
+
+# This is just to retrieve variables from Airflow Breeze in case it is available
+from tests.contrib.operators.test_gcp_sql_operator_system_helper \
+    import CloudSqlQueryTestHelper
+from tests.contrib.utils.gcp_authenticator import GCP_CLOUDSQL_KEY, GcpAuthenticator
+
+QUERY_SUFFIX = "_QUERY"
+
+if __name__ == '__main__':
+    parser = argparse.ArgumentParser(
+        description='Create or delete Cloud SQL instances for system tests.')
+    parser.add_argument('--action', required=True,
+                        choices=('create', 'delete', 'setup-instances',
+                                 'before-tests', 'after-tests'))
+    action = parser.parse_args().action
+
+    helper = CloudSqlQueryTestHelper()
+    gcp_authenticator = GcpAuthenticator(gcp_key=GCP_CLOUDSQL_KEY)
+    helper.log.info('Starting action: {}'.format(action))
+    gcp_authenticator.gcp_store_authentication()
+    try:
+        gcp_authenticator.gcp_authenticate()
+        if action == 'before-tests':
+            helper.create_instances(instance_suffix=QUERY_SUFFIX)
+            helper.setup_instances(instance_suffix=QUERY_SUFFIX)
+        elif action == 'after-tests':
+            helper.delete_instances(instance_suffix=QUERY_SUFFIX)
+        elif action == 'create':
+            helper.create_instances(instance_suffix=QUERY_SUFFIX)
+        elif action == 'delete':
+            helper.delete_instances(instance_suffix=QUERY_SUFFIX)
+        elif action == 'setup-instances':
+            helper.setup_instances(instance_suffix=QUERY_SUFFIX)
+        else:
+            raise Exception("Unknown action: {}".format(action))
+    finally:
+        gcp_authenticator.gcp_restore_authentication()
+
+    helper.log.info('Finishing action: {}'.format(action))
diff --git a/tests/contrib/operators/test_gcs_acl_operator.py b/tests/contrib/operators/test_gcs_acl_operator.py
index 562c653574..f428397e8b 100644
--- a/tests/contrib/operators/test_gcs_acl_operator.py
+++ b/tests/contrib/operators/test_gcs_acl_operator.py
@@ -22,8 +22,6 @@
 from airflow.contrib.operators.gcs_acl_operator import \
     GoogleCloudStorageBucketCreateAclEntryOperator, \
     GoogleCloudStorageObjectCreateAclEntryOperator
-from tests.contrib.operators.test_gcp_base import BaseGcpIntegrationTestCase, \
-    SKIP_TEST_WARNING, GCP_GCS_KEY
 
 try:
     from unittest import mock
@@ -72,16 +70,3 @@ def test_object_create_acl(self, mock_hook):
             generation="test-generation",
             user_project="test-user-project"
         )
-
-
-@unittest.skipIf(
-    BaseGcpIntegrationTestCase.skip_check(GCP_GCS_KEY), SKIP_TEST_WARNING)
-class CloudStorageExampleDagsIntegrationTest(BaseGcpIntegrationTestCase):
-    def __init__(self, method_name='runTest'):
-        super(CloudStorageExampleDagsIntegrationTest, self).__init__(
-            method_name,
-            dag_id='example_gcs_acl',
-            gcp_key=GCP_GCS_KEY)
-
-    def test_run_example_dag_gcs_acl(self):
-        self._run_dag()
diff --git a/tests/contrib/operators/test_gcs_acl_operator_system.py b/tests/contrib/operators/test_gcs_acl_operator_system.py
new file mode 100644
index 0000000000..0afddcbc9e
--- /dev/null
+++ b/tests/contrib/operators/test_gcs_acl_operator_system.py
@@ -0,0 +1,36 @@
+# -*- coding: utf-8 -*-
+#
+# 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 unittest
+
+from tests.contrib.utils.base_gcp_system_test_case import \
+    SKIP_TEST_WARNING, DagGcpSystemTestCase
+from tests.contrib.utils.gcp_authenticator import GCP_GCS_KEY
+
+
+@unittest.skipIf(
+    DagGcpSystemTestCase.skip_check(GCP_GCS_KEY), SKIP_TEST_WARNING)
+class CloudStorageExampleDagsSystemTest(DagGcpSystemTestCase):
+    def __init__(self, method_name='runTest'):
+        super(CloudStorageExampleDagsSystemTest, self).__init__(
+            method_name,
+            dag_id='example_gcs_acl',
+            gcp_key=GCP_GCS_KEY)
+
+    def test_run_example_dag_gcs_acl(self):
+        self._run_dag()
diff --git a/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py b/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py
index 0825364884..2eeae3a8a9 100644
--- a/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py
+++ b/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py
@@ -35,7 +35,7 @@
 TASK_ID = 'test-s3-gcs-transfer-operator'
 S3_BUCKET = 'test-s3-bucket'
 GCS_BUCKET = 'test-gcs-bucket'
-PROJECT_ID = 'test-project'
+GCP_PROJECT_ID = 'test-project'
 DESCRIPTION = 'test-description'
 ACCESS_KEY = 'test-access-key'
 SECRET_KEY = 'test-secret-key'
@@ -57,7 +57,7 @@ def test_constructor(self):
             task_id=TASK_ID,
             s3_bucket=S3_BUCKET,
             gcs_bucket=GCS_BUCKET,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             description=DESCRIPTION,
             schedule=SCHEDULE,
         )
@@ -65,7 +65,7 @@ def test_constructor(self):
         self.assertEqual(operator.task_id, TASK_ID)
         self.assertEqual(operator.s3_bucket, S3_BUCKET)
         self.assertEqual(operator.gcs_bucket, GCS_BUCKET)
-        self.assertEqual(operator.project_id, PROJECT_ID)
+        self.assertEqual(operator.project_id, GCP_PROJECT_ID)
         self.assertEqual(operator.description, DESCRIPTION)
         self.assertEqual(operator.schedule, SCHEDULE)
 
@@ -78,7 +78,7 @@ def test_execute(self, mock_s3_hook, mock_transfer_hook):
             task_id=TASK_ID,
             s3_bucket=S3_BUCKET,
             gcs_bucket=GCS_BUCKET,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             description=DESCRIPTION,
             schedule=SCHEDULE,
         )
@@ -91,7 +91,7 @@ def test_execute(self, mock_s3_hook, mock_transfer_hook):
         operator.execute(None)
 
         mock_transfer_hook.return_value.create_transfer_job.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             description=DESCRIPTION,
             schedule=SCHEDULE,
             transfer_spec={
@@ -123,7 +123,7 @@ def test_execute_skip_wait(self, mock_s3_hook, mock_transfer_hook):
             task_id=TASK_ID,
             s3_bucket=S3_BUCKET,
             gcs_bucket=GCS_BUCKET,
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             description=DESCRIPTION,
             wait=False,
         )
@@ -136,7 +136,7 @@ def test_execute_skip_wait(self, mock_s3_hook, mock_transfer_hook):
         operator.execute(None)
 
         mock_transfer_hook.return_value.create_transfer_job.assert_called_once_with(
-            project_id=PROJECT_ID,
+            project_id=GCP_PROJECT_ID,
             description=DESCRIPTION,
             schedule=None,
             transfer_spec={
diff --git a/tests/contrib/utils/base_gcp_system_test_case.py b/tests/contrib/utils/base_gcp_system_test_case.py
new file mode 100644
index 0000000000..6ab62d1bfd
--- /dev/null
+++ b/tests/contrib/utils/base_gcp_system_test_case.py
@@ -0,0 +1,256 @@
+# -*- coding: utf-8 -*-
+#
+# 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 os
+import subprocess
+import unittest
+from glob import glob
+from shutil import move
+from tempfile import mkdtemp
+
+from airflow.utils import db as db_utils
+from airflow import models, settings, AirflowException, LoggingMixin
+from airflow.utils.timezone import datetime
+from tests.contrib.utils.gcp_authenticator import GcpAuthenticator
+from tests.contrib.utils.run_once_decorator import run_once
+
+AIRFLOW_MAIN_FOLDER = os.path.realpath(os.path.join(
+    os.path.dirname(os.path.realpath(__file__)),
+    os.pardir, os.pardir, os.pardir))
+
+AIRFLOW_BREEZE_FOLDER = os.path.realpath(os.path.join(AIRFLOW_MAIN_FOLDER,
+                                                      os.pardir, os.pardir, os.pardir))
+ENV_FILE_RETRIEVER = os.path.join(AIRFLOW_BREEZE_FOLDER,
+                                  "get_system_test_environment_variables.py")
+
+
+# Retrieve environment variables from airflow breeze if it is used
+class RetrieveVariables:
+    @staticmethod
+    @run_once
+    def retrieve_variables():
+        if os.path.isfile(ENV_FILE_RETRIEVER):
+            if os.environ.get('AIRFLOW__CORE__UNIT_TEST_MODE'):
+                raise Exception("Please unset the AIRFLOW__CORE__UNIT_TEST_MODE")
+            variables = subprocess.check_output([ENV_FILE_RETRIEVER]).decode("utf-8")
+            print("Applying variables retrieved")
+            for line in variables.split("\n"):
+                try:
+                    variable, key = line.split("=")
+                except ValueError:
+                    continue
+                print("{}={}".format(variable, key))
+                os.environ[variable] = key
+
+
+RetrieveVariables.retrieve_variables()
+
+DEFAULT_DATE = datetime(2015, 1, 1)
+
+CONTRIB_OPERATORS_EXAMPLES_DAG_FOLDER = os.path.join(
+    AIRFLOW_MAIN_FOLDER, "airflow", "contrib", "example_dags")
+
+OPERATORS_EXAMPLES_DAG_FOLDER = os.path.join(
+    AIRFLOW_MAIN_FOLDER, "airflow", "example_dags")
+
+AIRFLOW_HOME = os.environ.get('AIRFLOW_HOME',
+                              os.path.join(os.path.expanduser('~'), 'airflow'))
+
+DAG_FOLDER = os.path.join(AIRFLOW_HOME, "dags")
+
+
+SKIP_TEST_WARNING = """
+The test is only run when the test is run in with GCP-system-tests enabled
+Airflow Breeze environment. You can enable it in one of two ways:
+
+* Set AIRFLOW_BREEZE_CONFIG_DIR environment variable to point to the airflow-breeze-config
+  directory
+* Make sure you run the tests in Airflow Breeze environment workspace where
+  airflow-breeze-config directory is checked out next to the airflow-breeze-config
+
+""".format(__file__)
+
+
+class BaseGcpSystemTestCase(unittest.TestCase, LoggingMixin):
+    def __init__(self,
+                 method_name,
+                 gcp_key,
+                 project_extra=None):
+        super(BaseGcpSystemTestCase, self).__init__(methodName=method_name)
+        self.gcp_authenticator = GcpAuthenticator(gcp_key=gcp_key,
+                                                  project_extra=project_extra)
+        self.setup_called = False
+
+    @staticmethod
+    def skip_check(key_name):
+        return GcpAuthenticator(key_name).full_key_path is None
+
+    def setUp(self):
+        self.gcp_authenticator.gcp_store_authentication()
+        self.gcp_authenticator.gcp_authenticate()
+        # We checked that authentication works. Ne we revoke it to make
+        # sure we are not relying on the default authentication
+        self.gcp_authenticator.gcp_revoke_authentication()
+        self.setup_called = True
+
+    # noinspection PyPep8Naming
+    def tearDown(self):
+        self.gcp_authenticator.gcp_restore_authentication()
+
+
+class DagGcpSystemTestCase(BaseGcpSystemTestCase):
+    def __init__(self,
+                 method_name,
+                 dag_id,
+                 gcp_key,
+                 dag_name=None,
+                 require_local_executor=False,
+                 example_dags_folder=CONTRIB_OPERATORS_EXAMPLES_DAG_FOLDER,
+                 project_extra=None):
+        super(DagGcpSystemTestCase, self).__init__(method_name=method_name,
+                                                   gcp_key=gcp_key,
+                                                   project_extra=project_extra)
+        self.dag_id = dag_id
+        self.dag_name = self.dag_id + '.py' if not dag_name else dag_name
+        self.example_dags_folder = example_dags_folder
+        self.require_local_executor = require_local_executor
+        self.temp_dir = None
+
+    @staticmethod
+    def _get_dag_folder():
+        return DAG_FOLDER
+
+    @staticmethod
+    def _get_files_to_link(path):
+        """
+        Returns all file names (note - file names not paths)
+        that have the same base name as the .py dag file (for example dag_name.sql etc.)
+        :param path: path to the dag file.
+        :return: list of files matching the base name
+        """
+        prefix, ext = os.path.splitext(path)
+        assert ext == '.py', "Dag name should be a .py file and is {} file".format(ext)
+        files_to_link = []
+        for file in glob(prefix + ".*"):
+            files_to_link.append(os.path.basename(file))
+        return files_to_link
+
+    def _symlink_dag_and_associated_files(self, remove=False):
+        target_folder = self._get_dag_folder()
+        source_path = os.path.join(self.example_dags_folder, self.dag_name)
+        for file_name in self._get_files_to_link(source_path):
+            source_path = os.path.join(self.example_dags_folder, file_name)
+            target_path = os.path.join(target_folder, file_name)
+            if remove:
+                try:
+                    self.log.info("Remove symlink: {} -> {} ".format(
+                        target_path, source_path))
+                    os.remove(target_path)
+                except OSError:
+                    pass
+            else:
+                if not os.path.exists(target_path):
+                    self.log.info("Symlink: {} -> {} ".format(target_path, source_path))
+                    os.symlink(source_path, target_path)
+                else:
+                    self.log.info("Symlink {} already exists. Not symlinking it.".
+                                  format(target_path))
+
+    def _store_dags_to_temporary_directory(self):
+        dag_folder = self._get_dag_folder()
+        self.temp_dir = mkdtemp()
+        self.log.info("Storing DAGS from {} to temporary directory {}".
+                      format(dag_folder, self.temp_dir))
+        try:
+            os.mkdir(dag_folder)
+        except OSError:
+            pass
+        for file in os.listdir(dag_folder):
+            move(os.path.join(dag_folder, file), os.path.join(self.temp_dir, file))
+
+    def _restore_dags_from_temporary_directory(self):
+        dag_folder = self._get_dag_folder()
+        self.log.info("Restoring DAGS to {} from temporary directory {}"
+                      .format(dag_folder, self.temp_dir))
+        for file in os.listdir(self.temp_dir):
+            move(os.path.join(self.temp_dir, file), os.path.join(dag_folder, file))
+
+    def _run_dag(self):
+        self.log.info("Attempting to run DAG: {}".format(self.dag_id))
+        if not self.setup_called:
+            raise AirflowException("Please make sure to call super.setUp() in your "
+                                   "test class!")
+        dag_folder = self._get_dag_folder()
+        dag_bag = models.DagBag(dag_folder=dag_folder, include_examples=False)
+        self.args = {'owner': 'airflow', 'start_date': DEFAULT_DATE}
+        dag = dag_bag.get_dag(self.dag_id)
+        if dag is None:
+            raise AirflowException(
+                "The Dag {} could not be found. It's either an import problem or "
+                "the dag {} was not symlinked to the DAGs folder. "
+                "The content of the {} folder is {}".
+                format(self.dag_id,
+                       self.dag_id + ".py",
+                       dag_folder,
+                       os.listdir(dag_folder)))
+        dag.clear(reset_dag_runs=True)
+        dag.run(ignore_first_depends_on_past=True, verbose=True)
+
+    @staticmethod
+    def _check_local_executor_setup():
+        postgres_path = os.path.realpath(os.path.join(
+            AIRFLOW_MAIN_FOLDER,
+            "tests", "contrib", "operators", "postgres_local_executor.cfg"))
+        if postgres_path != os.environ.get('AIRFLOW_CONFIG'):
+            raise AirflowException(
+                """
+Please set AIRFLOW_CONFIG variable to '{}'
+and make sure you have a Postgres server running locally and
+airflow/airflow.db database created.
+
+You can create the database via these commands:
+'createuser root'
+'createdb airflow/airflow.db`
+
+""".format(postgres_path))
+
+    # noinspection PyPep8Naming
+    def setUp(self):
+        if self.require_local_executor:
+            self._check_local_executor_setup()
+        try:
+            # We want to avoid random errors while database got reset - those
+            # Are apparently triggered by parser trying to parse DAGs while
+            # The tables are dropped. We move the dags temporarily out of the dags folder
+            # and move them back after reset
+            self._store_dags_to_temporary_directory()
+            try:
+                db_utils.resetdb(settings.RBAC)
+            finally:
+                self._restore_dags_from_temporary_directory()
+            self._symlink_dag_and_associated_files()
+            super(DagGcpSystemTestCase, self).setUp()
+
+        except Exception as e:
+            # In case of any error during setup - restore the authentication
+            self.gcp_authenticator.gcp_restore_authentication()
+            raise e
+
+    def tearDown(self):
+        self._symlink_dag_and_associated_files(remove=True)
+        super(DagGcpSystemTestCase, self).tearDown()
diff --git a/tests/contrib/utils/gcp_authenticator.py b/tests/contrib/utils/gcp_authenticator.py
new file mode 100644
index 0000000000..10621c1815
--- /dev/null
+++ b/tests/contrib/utils/gcp_authenticator.py
@@ -0,0 +1,208 @@
+# -*- coding: utf-8 -*-
+#
+# 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 json
+import os
+import subprocess
+
+from airflow import settings, AirflowException
+from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor
+
+from airflow.models.connection import Connection
+
+GCP_COMPUTE_KEY = 'gcp_compute.json'
+GCP_FUNCTION_KEY = 'gcp_function.json'
+GCP_CLOUDSQL_KEY = 'gcp_cloudsql.json'
+GCP_BIGTABLE_KEY = 'gcp_bigtable.json'
+GCP_SPANNER_KEY = 'gcp_spanner.json'
+GCP_GCS_KEY = 'gcp_gcs.json'
+
+KEYPATH_EXTRA = 'extra__google_cloud_platform__key_path'
+KEYFILE_DICT_EXTRA = 'extra__google_cloud_platform__keyfile_dict'
+SCOPE_EXTRA = 'extra__google_cloud_platform__scope'
+PROJECT_EXTRA = 'extra__google_cloud_platform__project'
+
+AIRFLOW_MAIN_FOLDER = os.path.realpath(os.path.join(
+    os.path.dirname(os.path.realpath(__file__)),
+    os.pardir, os.pardir, os.pardir))
+
+
+class GcpAuthenticator(LoggingCommandExecutor):
+    """
+    Manages authentication to Google Cloud Platform. It helps to manage
+    connection - it can authenticate with the gcp key name specified
+    """
+    original_account = None
+
+    def __init__(self, gcp_key, project_extra=None):
+        """
+        Initialises the authenticator.
+
+        :param gcp_key: name of the key to use for authentication (see GCP_*_KEY values)
+        :param project_extra: optional extra project parameter passed to google cloud
+               connection
+        """
+        super(GcpAuthenticator, self).__init__()
+        self.gcp_key = gcp_key
+        self.project_extra = project_extra
+        self.project_id = self.get_project_id()
+        self.full_key_path = None
+        self._set_key_path()
+
+    @staticmethod
+    def get_project_id():
+        return os.environ.get('GCP_PROJECT_ID')
+
+    def set_key_path_in_airflow_connection(self):
+        """
+        Set key path in 'google_cloud_default' connection to point to the full
+        key path
+        :return: None
+        """
+        session = settings.Session()
+        try:
+            conn = session.query(Connection).filter(
+                Connection.conn_id == 'google_cloud_default')[0]
+            extras = conn.extra_dejson
+            extras[KEYPATH_EXTRA] = self.full_key_path
+            if extras.get(KEYFILE_DICT_EXTRA):
+                del extras[KEYFILE_DICT_EXTRA]
+            extras[SCOPE_EXTRA] = 'https://www.googleapis.com/auth/cloud-platform'
+            extras[PROJECT_EXTRA] = self.project_extra if self.project_extra else \
+                self.project_id
+            conn.extra = json.dumps(extras)
+            session.commit()
+        except BaseException as ex:
+            self.log.info('Airflow DB Session error:' + str(ex))
+            session.rollback()
+            raise
+        finally:
+            session.close()
+
+    def set_dictionary_in_airflow_connection(self):
+        """
+        Set dictionary in 'google_cloud_default' connection to contain content
+        of the json service account file.
+        :return: None
+        """
+        session = settings.Session()
+        try:
+            conn = session.query(Connection).filter(
+                Connection.conn_id == 'google_cloud_default')[0]
+            extras = conn.extra_dejson
+            with open(self.full_key_path, "r") as path_file:
+                content = json.load(path_file)
+            extras[KEYFILE_DICT_EXTRA] = json.dumps(content)
+            if extras.get(KEYPATH_EXTRA):
+                del extras[KEYPATH_EXTRA]
+            extras[SCOPE_EXTRA] = 'https://www.googleapis.com/auth/cloud-platform'
+            extras[PROJECT_EXTRA] = self.project_extra
+            conn.extra = json.dumps(extras)
+            session.commit()
+        except BaseException as ex:
+            self.log.info('Airflow DB Session error:' + str(ex))
+            session.rollback()
+            raise
+        finally:
+            session.close()
+
+    def _set_key_path(self):
+        """
+        Sets full key path - if AIRFLOW_BREEZE_CONFIG_DIR points to absolute
+            directory, it tries to find the key in this directory. Otherwise it assumes
+            that Airflow is run in the Airflow Breeze - checked out directory so
+            it tries to find the key folder in the workspace's airflow-breeze-config
+            directory.
+        :param : name of the key file to find.
+        """
+        if "AIRFLOW_BREEZE_CONFIG_DIR" in os.environ:
+            airflow_breeze_config_dir = os.environ["AIRFLOW_BREEZE_CONFIG_DIR"]
+        else:
+            airflow_breeze_config_dir = os.path.join(AIRFLOW_MAIN_FOLDER,
+                                                     os.pardir,
+                                                     "airflow-breeze-config")
+        if not os.path.isdir(airflow_breeze_config_dir):
+            self.log.info("The {} is not a directory".format(airflow_breeze_config_dir))
+        key_dir = os.path.join(airflow_breeze_config_dir, "keys")
+        if not os.path.isdir(key_dir):
+            self.log.info("The {} is not a directory".format(key_dir))
+            return
+        key_path = os.path.join(key_dir, self.gcp_key)
+        if not os.path.isfile(key_path):
+            self.log.info("The {} is missing".format(key_path))
+        self.full_key_path = key_path
+
+    def _validate_key_set(self):
+        if self.full_key_path is None:
+            raise AirflowException("The gcp_key is not set!")
+        if not os.path.isfile(self.full_key_path):
+            raise AirflowException(
+                "The key {} could not be found. Please copy it to the {} path.".
+                format(self.gcp_key, self.full_key_path))
+
+    def gcp_authenticate(self):
+        """
+        Authenticate with service account specified via key name.
+        """
+        self._validate_key_set()
+        self.log.info("Setting the GCP key to {}".format(self.full_key_path))
+        # Checking if we can authenticate using service account credentials provided
+        self.execute_cmd(
+            ['gcloud', 'auth', 'activate-service-account',
+             '--key-file={}'.format(self.full_key_path),
+             '--project={}'.format(self.project_id)])
+        self.set_key_path_in_airflow_connection()
+
+    def gcp_revoke_authentication(self):
+        """
+        Change default authentication to none - which is not existing one.
+        """
+        self._validate_key_set()
+        self.log.info("Revoking authentication - setting it to none")
+        self.execute_cmd(
+            ['gcloud', 'config', 'get-value', 'account',
+             '--project={}'.format(self.project_id)])
+        self.execute_cmd(['gcloud', 'config', 'set', 'account', 'none',
+                          '--project={}'.format(self.project_id)])
+
+    def gcp_store_authentication(self):
+        """
+        Store authentication as it was originally so it can be restored and revoke
+        authentication.
+        """
+        self._validate_key_set()
+        if not GcpAuthenticator.original_account:
+            GcpAuthenticator.original_account = self.check_output(
+                ['gcloud', 'config', 'get-value', 'account',
+                 '--project={}'.format(self.project_id)]).decode('utf-8')
+            self.log.info("Storing account: to restore it later {}".format(
+                GcpAuthenticator.original_account))
+
+    def gcp_restore_authentication(self):
+        """
+        Restore authentication to the original one one.
+        """
+        self._validate_key_set()
+        if GcpAuthenticator.original_account:
+            self.log.info("Restoring original account stored: {}".
+                          format(GcpAuthenticator.original_account))
+            subprocess.call(['gcloud', 'config', 'set', 'account',
+                             GcpAuthenticator.original_account,
+                             '--project={}'.format(self.project_id)])
+        else:
+            self.log.info("Not restoring the original GCP account: it is not set")
diff --git a/tests/contrib/utils/logging_command_executor.py b/tests/contrib/utils/logging_command_executor.py
new file mode 100644
index 0000000000..e0833f4a77
--- /dev/null
+++ b/tests/contrib/utils/logging_command_executor.py
@@ -0,0 +1,56 @@
+# -*- coding: utf-8 -*-
+#
+# 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 os
+import subprocess
+
+from airflow import LoggingMixin, AirflowException
+
+
+class LoggingCommandExecutor(LoggingMixin):
+
+    def execute_cmd(self, cmd, silent=False):
+        if silent:
+            self.log.info("Executing in silent mode: '{}'".format(" ".join(cmd)))
+            with open(os.devnull, 'w') as FNULL:
+                return subprocess.call(args=cmd, stdout=FNULL, stderr=subprocess.STDOUT)
+        else:
+            self.log.info("Executing: '{}'".format(" ".join(cmd)))
+            process = subprocess.Popen(args=cmd, stdout=subprocess.PIPE,
+                                       stderr=subprocess.PIPE, universal_newlines=True)
+            output, err = process.communicate()
+            retcode = process.poll()
+            self.log.info("Stdout: {}".format(output))
+            self.log.info("Stderr: {}".format(err))
+            if retcode:
+                print("Error when executing '{}'".format(" ".join(cmd)))
+            return retcode
+
+    def check_output(self, cmd):
+        self.log.info("Executing for output: '{}'".format(" ".join(cmd)))
+        process = subprocess.Popen(args=cmd, stdout=subprocess.PIPE,
+                                   stderr=subprocess.PIPE)
+        output, err = process.communicate()
+        retcode = process.poll()
+        if retcode:
+            self.log.info("Error when executing '{}'".format(" ".join(cmd)))
+            self.log.info("Stdout: {}".format(output))
+            self.log.info("Stderr: {}".format(err))
+            raise AirflowException("Retcode {} on {} with stdout: {}, stderr: {}".
+                                   format(retcode, " ".join(cmd), output, err))
+        return output
diff --git a/tests/contrib/utils/run_once_decorator.py b/tests/contrib/utils/run_once_decorator.py
new file mode 100644
index 0000000000..934250ec2f
--- /dev/null
+++ b/tests/contrib/utils/run_once_decorator.py
@@ -0,0 +1,35 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+#
+
+from functools import wraps
+
+
+def run_once(f):
+    """Runs a function (successfully) only once.
+    The running can be reset by setting the `has_run` attribute to False
+    """
+    @wraps(f)
+    def wrapper(*args, **kwargs):
+        if not wrapper.has_run:
+            result = f(*args, **kwargs)
+            wrapper.has_run = True
+            return result
+    wrapper.has_run = False
+    return wrapper
diff --git a/tests/dags/.gitignore b/tests/dags/.gitignore
index d42b544171..37ab6d48e9 100644
--- a/tests/dags/.gitignore
+++ b/tests/dags/.gitignore
@@ -1,5 +1,4 @@
 # This line is to avoid accidental commits of example dags for integration testing
 # In order to test example dags easily we often create symbolic links in this directory
-# and run the Airflow with AIRFLOW__CORE__UNIT_TEST_MODE=True
 # this line prevents accidental committing of such symbolic links.
 example_*


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services