Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 0 additions & 2 deletions airflow-core/tests/unit/always/test_project_structure.py
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,6 @@ def test_providers_modules_should_have_tests(self):
"providers/google/tests/unit/google/cloud/links/test_dataproc.py",
"providers/google/tests/unit/google/cloud/links/test_datastore.py",
"providers/google/tests/unit/google/cloud/links/test_kubernetes_engine.py",
"providers/google/tests/unit/google/cloud/links/test_life_sciences.py",
"providers/google/tests/unit/google/cloud/links/test_mlengine.py",
"providers/google/tests/unit/google/cloud/links/test_pubsub.py",
"providers/google/tests/unit/google/cloud/links/test_spanner.py",
Expand Down Expand Up @@ -496,7 +495,6 @@ class TestGoogleProviderProjectStructure(ExampleCoverageTest, AssetsCoverageTest
}

MISSING_EXAMPLES_FOR_CLASSES = {
"airflow.providers.google.cloud.operators.life_sciences.LifeSciencesRunPipelineOperator",
"airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator",
"airflow.providers.google.cloud.transfers.cassandra_to_gcs.CassandraToGCSOperator",
"airflow.providers.google.cloud.transfers.adls_to_gcs.ADLSToGCSOperator",
Expand Down
24 changes: 24 additions & 0 deletions providers/google/docs/changelog.rst
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,30 @@
Changelog
---------

17.0.0
......

.. warning::
Deprecated classes, parameters and features have been removed from the Google provider package.
The following breaking changes were introduced:

* Operators

* ``Remove operator LifeSciencesRunPipelineOperator use Google Cloud Batch operators instead``
* ``Remove operator BigQueryCreateEmptyTableOperator use BigQueryCreateTableOperator instead``
* ``Remove operator BigQueryCreateExternalTableOperator use BigQueryCreateTableOperator instead``

* Hooks

* ``Remove LifeSciencesHook use Google Cloud Batch hooks instead``
* ``Remove BigQueryHook.create_empty_table method use BigQueryHook.create_table instead``
* ``Remove DataflowHook.start_sql_job method use DataflowHook.launch_beam_yaml_job instead``

* Links

* ``Remove LifeSciencesLink as support of the Life Sciences API will be discontinued on July 8, 2025``


16.1.0
......

Expand Down
26 changes: 0 additions & 26 deletions providers/google/docs/operators/cloud/bigquery.rst
Original file line number Diff line number Diff line change
Expand Up @@ -188,32 +188,6 @@ caches results of a query for increased performance and efficiency.
:start-after: [START howto_operator_bigquery_create_materialized_view]
:end-before: [END howto_operator_bigquery_create_materialized_view]

.. _howto/operator:BigQueryCreateEmptyTableOperator:

Create native table
"""""""""""""""""""

.. warning::
This operator is deprecated and will be removed after July 30, 2025. Please use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateTableOperator`.

To create a new, empty table in the given BigQuery dataset, optionally with
schema you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyTableOperator`.

.. _howto/operator:BigQueryCreateExternalTableOperator:

Create external table
"""""""""""""""""""""

.. warning::
This operator is deprecated and will be removed after July 30, 2025. Please use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateTableOperator`.

To create a new external table in a dataset with the data in Google Cloud Storage
you can use
:class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateExternalTableOperator`.

.. _howto/operator:BigQueryGetDataOperator:

Fetch data from table
Expand Down
58 changes: 0 additions & 58 deletions providers/google/docs/operators/cloud/life_sciences.rst

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ the BigQuery documentation.

Airflow also has numerous operators that allow you to create the use of BigQuery.
For example, if you want to create an external table that allows you to create queries that
read data directly from GCS, then you can use :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateExternalTableOperator`.
read data directly from GCS, then you can use :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateTableOperator`.
Using this operator looks like this:

.. exampleinclude:: /../../google/tests/system/google/cloud/gcs/example_presto_to_gcs.py
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ the BigQuery documentation.

Airflow also has numerous operators that allow you to create the use of BigQuery.
For example, if you want to create an external table that allows you to create queries that
read data directly from GCS, then you can use :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateExternalTableOperator`.
read data directly from GCS, then you can use :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateTableOperator`.
Using this operator looks like this:

.. exampleinclude:: /../../google/tests/system/google/cloud/gcs/example_trino_to_gcs.py
Expand Down
13 changes: 0 additions & 13 deletions providers/google/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -203,12 +203,6 @@ integrations:
external-doc-url: https://cloud.google.com/security-key-management/
logo: /docs/integration-logos/Key-Management-Service.png
tags: [gcp]
- integration-name: Google Cloud Life Sciences
external-doc-url: https://cloud.google.com/life-sciences/
how-to-guide:
- /docs/apache-airflow-providers-google/operators/cloud/life_sciences.rst
logo: /docs/integration-logos/Google-Cloud-Life-Sciences.png
tags: [gcp]
- integration-name: Google Cloud Managed Service for Apache Kafka
external-doc-url: https://cloud.google.com/managed-service-for-apache-kafka/docs/
how-to-guide:
Expand Down Expand Up @@ -534,9 +528,6 @@ operators:
- integration-name: Google Kubernetes Engine
python-modules:
- airflow.providers.google.cloud.operators.kubernetes_engine
- integration-name: Google Cloud Life Sciences
python-modules:
- airflow.providers.google.cloud.operators.life_sciences
- integration-name: Google Machine Learning Engine
python-modules:
- airflow.providers.google.cloud.operators.mlengine
Expand Down Expand Up @@ -800,9 +791,6 @@ hooks:
- integration-name: Google Kubernetes Engine
python-modules:
- airflow.providers.google.cloud.hooks.kubernetes_engine
- integration-name: Google Cloud Life Sciences
python-modules:
- airflow.providers.google.cloud.hooks.life_sciences
- integration-name: Google Machine Learning Engine
python-modules:
- airflow.providers.google.cloud.hooks.mlengine
Expand Down Expand Up @@ -1214,7 +1202,6 @@ extra-links:
- airflow.providers.google.cloud.links.cloud_build.CloudBuildListLink
- airflow.providers.google.cloud.links.cloud_build.CloudBuildTriggersListLink
- airflow.providers.google.cloud.links.cloud_build.CloudBuildTriggerDetailsLink
- airflow.providers.google.cloud.links.life_sciences.LifeSciencesLink
- airflow.providers.google.cloud.links.cloud_functions.CloudFunctionsDetailsLink
- airflow.providers.google.cloud.links.cloud_functions.CloudFunctionsListLink
- airflow.providers.google.cloud.links.cloud_storage_transfer.CloudStorageTransferListLink
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -400,135 +400,6 @@ def table_partition_exists(
except NotFound:
return False

@deprecated(
planned_removal_date="July 30, 2025",
use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_table",
category=AirflowProviderDeprecationWarning,
)
@GoogleBaseHook.fallback_to_default_project_id
def create_empty_table(
self,
project_id: str = PROVIDE_PROJECT_ID,
dataset_id: str | None = None,
table_id: str | None = None,
table_resource: dict[str, Any] | None = None,
schema_fields: list | None = None,
time_partitioning: dict | None = None,
cluster_fields: list[str] | None = None,
labels: dict | None = None,
view: dict | None = None,
materialized_view: dict | None = None,
encryption_configuration: dict | None = None,
retry: Retry = DEFAULT_RETRY,
location: str | None = None,
exists_ok: bool = True,
) -> Table:
"""
Create a new, empty table in the dataset.

To create a view, which is defined by a SQL query, parse a dictionary to
the *view* argument.

:param project_id: The project to create the table into.
:param dataset_id: The dataset to create the table into.
:param table_id: The Name of the table to be created.
:param table_resource: Table resource as described in documentation:
https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#Table
If provided all other parameters are ignored.
:param schema_fields: If set, the schema field list as defined here:
https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema

.. code-block:: python

schema_fields = [
{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
{"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
]

:param labels: a dictionary containing labels for the table, passed to BigQuery
:param retry: Optional. How to retry the RPC.
:param time_partitioning: configure optional time partitioning fields i.e.
partition by field, type and expiration as per API specifications.

.. seealso::
https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#timePartitioning
:param cluster_fields: [Optional] The fields used for clustering.
BigQuery supports clustering for both partitioned and
non-partitioned tables.
https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#clustering.fields
:param view: [Optional] A dictionary containing definition for the view.
If set, it will create a view instead of a table:
https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#ViewDefinition

.. code-block:: python

view = {
"query": "SELECT * FROM `test-project-id.test_dataset_id.test_table_prefix*` LIMIT 1000",
"useLegacySql": False,
}

:param materialized_view: [Optional] The materialized view definition.
:param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys).

.. code-block:: python

encryption_configuration = {
"kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key",
}

:param num_retries: Maximum number of retries in case of connection problems.
:param location: (Optional) The geographic location where the table should reside.
:param exists_ok: If ``True``, ignore "already exists" errors when creating the table.
:return: Created table
"""
_table_resource: dict[str, Any] = {}

if self.location:
_table_resource["location"] = self.location

if schema_fields:
_table_resource["schema"] = {"fields": schema_fields}

if time_partitioning:
_table_resource["timePartitioning"] = time_partitioning

if cluster_fields:
_table_resource["clustering"] = {"fields": cluster_fields}

if labels:
_table_resource["labels"] = labels

if view:
_table_resource["view"] = view

if materialized_view:
_table_resource["materializedView"] = materialized_view

if encryption_configuration:
_table_resource["encryptionConfiguration"] = encryption_configuration

table_resource = table_resource or _table_resource
table_resource = self._resolve_table_reference(
table_resource=table_resource,
project_id=project_id,
dataset_id=dataset_id,
table_id=table_id,
)
table = Table.from_api_repr(table_resource)
result = self.get_client(project_id=project_id, location=location).create_table(
table=table, exists_ok=exists_ok, retry=retry
)
get_hook_lineage_collector().add_output_asset(
context=self,
scheme="bigquery",
asset_kwargs={
"project_id": result.project,
"dataset_id": result.dataset_id,
"table_id": result.table_id,
},
)
return result

@GoogleBaseHook.fallback_to_default_project_id
def create_table(
self,
Expand Down
Loading