Skip to content

Commit

Permalink
Deprecate only unusable operators and hooks
Browse files Browse the repository at this point in the history
  • Loading branch information
shahar1 committed Mar 30, 2024
1 parent b435116 commit 653e838
Show file tree
Hide file tree
Showing 10 changed files with 256 additions and 1,264 deletions.
153 changes: 18 additions & 135 deletions airflow/providers/google/cloud/hooks/automl.py
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
BatchPredictOutputConfig,
Dataset,
ExamplePayload,
ImageObjectDetectionModelDeploymentMetadata,
InputConfig,
Model,
PredictionServiceClient,
Expand All @@ -50,9 +49,7 @@
from google.api_core.operation import Operation
from google.api_core.retry import Retry
from google.cloud.automl_v1beta1.services.auto_ml.pagers import (
ListColumnSpecsPager,
ListDatasetsPager,
ListTableSpecsPager,
)
from google.protobuf.field_mask_pb2 import FieldMask

Expand Down Expand Up @@ -321,58 +318,16 @@ def import_data(
)
return result

@GoogleBaseHook.fallback_to_default_project_id
def list_column_specs(
self,
dataset_id: str,
table_spec_id: str,
location: str,
project_id: str = PROVIDE_PROJECT_ID,
field_mask: dict | FieldMask | None = None,
filter_: str | None = None,
page_size: int | None = None,
retry: Retry | _MethodDefault = DEFAULT,
timeout: float | None = None,
metadata: Sequence[tuple[str, str]] = (),
) -> ListColumnSpecsPager:
def list_column_specs(self, **kwargs) -> None:
"""
List column specs in a table spec.
:param dataset_id: Name of the AutoML dataset.
:param table_spec_id: table_spec_id for path builder.
:param field_mask: Mask specifying which fields to read. If a dict is provided, it must be of the same
form as the protobuf message `google.cloud.automl_v1beta1.types.FieldMask`
:param filter_: Filter expression, see go/filtering.
:param page_size: The maximum number of resources contained in the
underlying API response. If page streaming is performed per
resource, this parameter does not affect the return value. If page
streaming is performed per-page, this determines the maximum number
of resources in a page.
:param project_id: ID of the Google Cloud project where dataset is located if None then
default project_id is used.
:param location: The location of the project.
:param retry: A retry object used to retry requests. If `None` is specified, requests will not be
retried.
:param timeout: The amount of time, in seconds, to wait for the request to complete. Note that if
`retry` is specified, the timeout applies to each individual attempt.
:param metadata: Additional metadata that is provided to the method.
List column specs in a table spec (Deprecated).
:return: `google.cloud.automl_v1beta1.types.ColumnSpec` instance.
:raises: AirflowException
"""
client = self.get_conn()
parent = client.table_spec_path(
project=project_id,
location=location,
dataset=dataset_id,
table_spec=table_spec_id,
raise AirflowException(
"This method is deprecated as corresponding API becomes no longer available. See:"
"https://cloud.google.com/automl/docs/reference/rest/v1beta1/projects.locations.datasets.tableSpecs.columnSpecs/list"
)
result = client.list_column_specs(
request={"parent": parent, "field_mask": field_mask, "filter": filter_, "page_size": page_size},
retry=retry,
timeout=timeout,
metadata=metadata,
)
return result

@GoogleBaseHook.fallback_to_default_project_id
def get_model(
Expand Down Expand Up @@ -476,99 +431,27 @@ def update_dataset(
)
return result

@GoogleBaseHook.fallback_to_default_project_id
def deploy_model(
self,
model_id: str,
location: str,
project_id: str = PROVIDE_PROJECT_ID,
image_detection_metadata: ImageObjectDetectionModelDeploymentMetadata | dict | None = None,
retry: Retry | _MethodDefault = DEFAULT,
timeout: float | None = None,
metadata: Sequence[tuple[str, str]] = (),
) -> Operation:
def deploy_model(self, **kwargs) -> None:
"""
Deploys a model.
If a model is already deployed, deploying it with the same parameters
has no effect. Deploying with different parameters (as e.g. changing node_number) will
reset the deployment state without pausing the model_id's availability.
Only applicable for Text Classification, Image Object Detection and Tables; all other
domains manage deployment automatically.
Deploys a model (Deprecated).
:param model_id: Name of the model requested to serve the prediction.
:param image_detection_metadata: Model deployment metadata specific to Image Object Detection.
If a dict is provided, it must be of the same form as the protobuf message
ImageObjectDetectionModelDeploymentMetadata
:param project_id: ID of the Google Cloud project where model will be created if None then
default project_id is used.
:param location: The location of the project.
:param retry: A retry object used to retry requests. If `None` is specified, requests will not be
retried.
:param timeout: The amount of time, in seconds, to wait for the request to complete. Note that if
`retry` is specified, the timeout applies to each individual attempt.
:param metadata: Additional metadata that is provided to the method.
:return: `google.cloud.automl_v1beta1.types._OperationFuture` instance.
:raises: AirflowException
"""
client = self.get_conn()
name = f"projects/{project_id}/locations/{location}/models/{model_id}"
result = client.deploy_model(
request={
"name": name,
"image_object_detection_model_deployment_metadata": image_detection_metadata,
},
retry=retry,
timeout=timeout,
metadata=metadata,
raise AirflowException(
"This method is deprecated as corresponding API becomes no longer available. See:"
"https://cloud.google.com/automl/docs/reference/rest/v1beta1/projects.locations.models/deploy "
)
return result

def list_table_specs(
self,
dataset_id: str,
location: str,
project_id: str | None = None,
filter_: str | None = None,
page_size: int | None = None,
retry: Retry | _MethodDefault = DEFAULT,
timeout: float | None = None,
metadata: Sequence[tuple[str, str]] = (),
) -> ListTableSpecsPager:
def list_table_specs(self, **kwargs) -> None:
"""
List table specs in a dataset_id.
:param dataset_id: Name of the dataset.
:param filter_: Filter expression, see go/filtering.
:param page_size: The maximum number of resources contained in the
underlying API response. If page streaming is performed per
resource, this parameter does not affect the return value. If page
streaming is performed per-page, this determines the maximum number
of resources in a page.
:param project_id: ID of the Google Cloud project where dataset is located if None then
default project_id is used.
:param location: The location of the project.
:param retry: A retry object used to retry requests. If `None` is specified, requests will not be
retried.
:param timeout: The amount of time, in seconds, to wait for the request to complete. Note that if
`retry` is specified, the timeout applies to each individual attempt.
:param metadata: Additional metadata that is provided to the method.
List table specs in a dataset_id (Deprecated).
:return: A `google.gax.PageIterator` instance. By default, this
is an iterable of `google.cloud.automl_v1beta1.types.TableSpec` instances.
This object can also be configured to iterate over the pages
of the response through the `options` parameter.
:raises: AirflowException
"""
client = self.get_conn()
parent = f"projects/{project_id}/locations/{location}/datasets/{dataset_id}"
result = client.list_table_specs(
request={"parent": parent, "filter": filter_, "page_size": page_size},
retry=retry,
timeout=timeout,
metadata=metadata,
raise AirflowException(
"This method is deprecated as corresponding API becomes no longer available. See:"
"https://cloud.google.com/automl/docs/reference/rest/v1beta1/projects.locations.datasets.tableSpecs/list "
)
return result

@GoogleBaseHook.fallback_to_default_project_id
def list_datasets(
Expand Down
Loading

0 comments on commit 653e838

Please sign in to comment.