You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "PierreC1024 (via GitHub)" <gi...@apache.org> on 2023/03/01 11:08:25 UTC

[GitHub] [airflow] PierreC1024 opened a new pull request, #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

PierreC1024 opened a new pull request, #29835:
URL: https://github.com/apache/airflow/pull/29835

   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   **In Google Cloud Provider, adding BigQuery BI Engine reservation operators.**
   
   [BI Engine](https://cloud.google.com/bigquery/docs/bi-engine-intro) is a fast, in-memory analysis service that accelerates many SQL queries in BigQuery by intelligently caching the data you use most frequently. Mainly, it is used for BI tools on top of BigQuery. Because BI tools has been used mainly during the working hours, you could save BI Engine cost to use it only during specific time slots.
   
   The two operators added`BigQueryBiEngineReservationCreateOperator` and `BigQueryBiEngineReservationDeleteOperator` allows to create and delete BI Engine reservation. 
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1126766827


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"

Review Comment:
   You are right, it is not required. I delete `default_conn_name ` and `conn_name_attr` the default value will be normally taken by the inheritance.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1127689746


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,123 @@
+#

Review Comment:
   I have thought about it but it uses the bigquery-reservation API. So, I would prefer keep this name. Moreover, this could be extended with other bigquery-reservation api routes like create and delete bigquery slot reservation (so using the same client). 
   
   PS: To be honest, It should be my next PR (create and delete BigQuery slots reservation operators) but I wish to validate all the process with BI Engine first. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1126311116


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"

Review Comment:
   I would prefer to use `google_cloud_default` name and not introduce a new one unless it absolutely requires. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on PR #29835:
URL: https://github.com/apache/airflow/pull/29835#issuecomment-1456537606

   @pankajastro. Thanks for your review and your approuval.
   What is the next steps to merge PR and release the new google provider version ? Do I have something to do on my side ? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] commented on pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #29835:
URL: https://github.com/apache/airflow/pull/29835#issuecomment-1518902017

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1127703445


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,123 @@
+#

Review Comment:
   However, I am open, if you prefer this naming, I will do it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1127679850


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,123 @@
+#

Review Comment:
   I wonder if it's better to name it `hooks/bigquery_biengine.py` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1122930562


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,129 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param delegate_to: This performs a task on one host with reference to other hosts.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    conn_type = "gcpbigqueryreservation"

Review Comment:
   We need a new connection to respect minimum privileges. 
   Indeed, the connection type could be `google_cloud_platform`.  I delete it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1123625999


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   I try to do like this but currently it is not possible to define `project_id` as an attribute. 
   This is because of `GoogleBaseHook` have defined `project_id` with a decorated @property and it does not have a setter method for `project_id` ([see](https://github.com/apache/airflow/blob/main/airflow/providers/google/common/hooks/base_google.py#L342)).  
   
   I am not confortable to change it in this PR. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1123530878


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   do we need project_id args here? can we access this with `self.project_id` since this is a base class property? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1122855854


##########
airflow/providers/google/cloud/example_dags/example_bigquery_reservation_bi_engine.py:
##########
@@ -0,0 +1,72 @@
+#
+# 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 performs BI Engine.
+
+This DAG reserves a 100GB BI engine reservation between from 8am ato 7pm each working days.
+This DAG executes this following workflow:
+    1. Create BI reservation
+    2. Wait the precise date
+    3. Delete BI reservation
+"""
+from __future__ import annotations
+
+import os
+
+from pendulum import Time, datetime
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery_reservation import (
+    BigQueryBiEngineReservationCreateOperator,
+    BigQueryBiEngineReservationDeleteOperator,
+)
+from airflow.sensors.time_sensor import TimeSensor
+
+END_TIME = Time(19, 0, 0)
+BI_ENGINE_RESERVATION_SIZE = 100
+PROJECT_ID = os.getenv("GCP_PROJECT_ID", "example-project")
+LOCATION = os.getenv("GCP_PROJECT_LOCATION", "US")
+
+with models.DAG(

Review Comment:
   Yes you're right, it is not necessary to have both.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1122802132


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,129 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param delegate_to: This performs a task on one host with reference to other hosts.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    conn_type = "gcpbigqueryreservation"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        delegate_to: str | None = None,

Review Comment:
   in `GoogleBaseHook` we have deprecated `delegate_to` so can we remove this from here?



##########
airflow/providers/google/cloud/operators/bigquery_reservation.py:
##########
@@ -0,0 +1,150 @@
+#
+# 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 module contains Google BigQuery Reservation Service operators."""
+from __future__ import annotations
+
+from typing import Any, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.google.cloud.hooks.bigquery_reservation import BigQueryReservationServiceHook
+
+
+class BigQueryBiEngineReservationCreateOperator(BaseOperator):
+    """
+    Create or Update BI engine reservation.
+
+    :param location: The BI engine reservation location. (templated)
+    :param size: The BI Engine reservation memory size (GB). (templated)
+    :param project_id: (Optional) The name of the project where the reservation
+        will be attached from. (templated)
+    :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. (templated)
+    :param delegate_to: Account to impersonate using domain-wide delegation of authority,

Review Comment:
   since we have deprecated `delegate_to` in google provider so can avoid adding it here?



##########
airflow/providers/google/cloud/example_dags/example_bigquery_reservation_bi_engine.py:
##########
@@ -0,0 +1,72 @@
+#
+# 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 performs BI Engine.
+
+This DAG reserves a 100GB BI engine reservation between from 8am ato 7pm each working days.
+This DAG executes this following workflow:
+    1. Create BI reservation
+    2. Wait the precise date
+    3. Delete BI reservation
+"""
+from __future__ import annotations
+
+import os
+
+from pendulum import Time, datetime
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery_reservation import (
+    BigQueryBiEngineReservationCreateOperator,
+    BigQueryBiEngineReservationDeleteOperator,
+)
+from airflow.sensors.time_sensor import TimeSensor
+
+END_TIME = Time(19, 0, 0)
+BI_ENGINE_RESERVATION_SIZE = 100
+PROJECT_ID = os.getenv("GCP_PROJECT_ID", "example-project")
+LOCATION = os.getenv("GCP_PROJECT_LOCATION", "US")
+
+with models.DAG(

Review Comment:
   not sure if we should move this to system tests?



##########
tests/providers/google/cloud/operators/test_bigquery_reservation.py:
##########
@@ -0,0 +1,71 @@
+#
+# 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 __future__ import annotations
+
+import unittest
+from unittest import mock
+
+from airflow.providers.google.cloud.operators.bigquery_reservation import (
+    BigQueryBiEngineReservationCreateOperator,
+    BigQueryBiEngineReservationDeleteOperator,
+)
+
+TASK_ID = "id"
+PROJECT_ID = "test-project"
+LOCATION = "US"
+SIZE = 100
+
+
+class TestBigQueryBiEngineReservationCreateOperator(unittest.TestCase):

Review Comment:
   ```suggestion
   class TestBigQueryBiEngineReservationCreateOperator:
   ```



##########
airflow/providers/google/cloud/example_dags/example_bigquery_reservation_bi_engine.py:
##########
@@ -0,0 +1,72 @@
+#
+# 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 performs BI Engine.
+
+This DAG reserves a 100GB BI engine reservation between from 8am ato 7pm each working days.
+This DAG executes this following workflow:
+    1. Create BI reservation
+    2. Wait the precise date
+    3. Delete BI reservation
+"""
+from __future__ import annotations
+
+import os
+
+from pendulum import Time, datetime
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery_reservation import (
+    BigQueryBiEngineReservationCreateOperator,
+    BigQueryBiEngineReservationDeleteOperator,
+)
+from airflow.sensors.time_sensor import TimeSensor
+
+END_TIME = Time(19, 0, 0)
+BI_ENGINE_RESERVATION_SIZE = 100
+PROJECT_ID = os.getenv("GCP_PROJECT_ID", "example-project")
+LOCATION = os.getenv("GCP_PROJECT_LOCATION", "US")
+
+with models.DAG(

Review Comment:
   ohh you have already added the system test then do we need example here too?



##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,129 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param delegate_to: This performs a task on one host with reference to other hosts.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    conn_type = "gcpbigqueryreservation"

Review Comment:
   do we need a new connection type?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on PR #29835:
URL: https://github.com/apache/airflow/pull/29835#issuecomment-1457604573

   > @pankajastro. Thanks for your review and your approuval. What is the next steps to merge PR and release the new google provider version ? Do I have something to do on my side ?
   
   CI is failing and you may need to fix that but it looks flaky so you can just try to do a force push. `git commit --amend` then `git push -f`. once CI is green someone from the community having write access can merge it. additionally, if you want you can post it in the community slack #development channel if that help. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1127744289


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,123 @@
+#

Review Comment:
   Thanks for your detailed answer, I will change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] closed pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators
URL: https://github.com/apache/airflow/pull/29835


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1126307197


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   I see - then I'm ok to keep it in its current form. thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1123543321


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   Most of the time you wish to create bi_engine reservation not on the same project than the default project (defining in the connection). 
   I choose to put it as the function argument but we could add it in the `BigQueryReservationServiceHook` attribute. What do you prefer ?    



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1124427992


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   I take a look to other Google hook, most of them have`project_id` as an arguments of functions. 
   Currently, I propose to you to stay like this. 
   But, perahps open an issue and a new PR to change this. 
   Do you agree ? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1129320448


##########
airflow/providers/google/cloud/hooks/bigquery_biengine.py:
##########
@@ -0,0 +1,123 @@
+#
+# 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 module contains a BigQuery BI Engine Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Bi Engine reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    hook_name = "Google Bigquery BI Engine"

Review Comment:
   Don't we need to set also `conn_name_attr`, `default_conn_name`, `conn_type`
   For example:
   https://github.com/apache/airflow/blob/c3867781e09b7e0e0d19c0991865a2453194d9a8/airflow/providers/google/cloud/hooks/bigquery.py#L85-L88
   
   do we want users to use BigQuery conn for this hook or have a separated one?



##########
airflow/providers/google/cloud/hooks/bigquery_biengine.py:
##########
@@ -0,0 +1,123 @@
+#
+# 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 module contains a BigQuery BI Engine Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Bi Engine reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    hook_name = "Google Bigquery BI Engine"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:
+        """
+        Create BI Engine reservation
+
+        :param project_id: The name of the project where we want to create/update
+            the BI Engine reservation.
+        :param size: The BI Engine reservation size in gigabyte.
+        """
+        parent = f"projects/{project_id}/locations/{self.location}/biReservation"
+        client = self.get_client()
+        size = self._convert_gb_to_kb(value=size)
+
+        try:
+            bi_reservation = client.get_bi_reservation(name=parent)
+            bi_reservation.size = size + bi_reservation.size
+
+            client.update_bi_reservation(bi_reservation=bi_reservation)
+
+            self.log.info("BI Engine reservation {parent} have been updated to {bi_reservation.size}Kb.")
+        except Exception as e:
+            self.log.error(e)
+            raise AirflowException(f"Failed to create BI engine reservation of {size}.")
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def delete_bi_reservation(self, project_id: str, size: int | None = None) -> None:
+        """
+        Delete/Update BI Engine reservation with the specified memory size
+
+        :param project_id: The name of the project where we want to delete/update
+            the BI Engine reservation.
+        :param size: The BI Engine reservation size in gigabyte.
+        """
+        parent = f"projects/{project_id}/locations/{self.location}/biReservation"
+        client = self.get_client()
+        try:
+            bi_reservation = client.get_bi_reservation(name=parent)
+            if size is not None:
+                size = self._convert_gb_to_kb(size)
+                bi_reservation.size = max(bi_reservation.size - size, 0)
+            else:
+                bi_reservation.size = 0
+
+            client.update_bi_reservation(bi_reservation=bi_reservation)
+            self.log.info("BI Engine reservation {parent} have been updated to {bi_reservation.size}Kb.")
+        except Exception as e:

Review Comment:
   This is very broad.
   We should catch only the relevant exception
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1123543321


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   Sometimes, you wish to create bi_engine reservation not on the same project than the default project (defining in the connection). 
   I choose to put it as the function argument but we could add it in the `BigQueryReservationServiceHook` attribute. What do you prefer ?    



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1123551136


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   hmm, in that case I would prefer to have this in `BigQueryReservationServiceHook`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] PierreC1024 commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "PierreC1024 (via GitHub)" <gi...@apache.org>.
PierreC1024 commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1123625999


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,125 @@
+#
+# 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 module contains a BigQuery Reservation Hook."""
+from __future__ import annotations
+
+from typing import Sequence
+
+from google.cloud.bigquery_reservation_v1 import ReservationServiceClient
+
+from airflow.exceptions import AirflowException
+from airflow.providers.google.common.consts import CLIENT_INFO
+from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
+
+
+class BigQueryReservationServiceHook(GoogleBaseHook):
+    """
+    Hook for Google Bigquery Reservation API.
+
+    :param gcp_conn_id: The Airflow connection used for GCP credentials.
+    :param location: The location of the BigQuery resource.
+    :param impersonation_chain: This is the optional service account to impersonate using short term
+        credentials.
+    """
+
+    conn_name_attr = "gcp_conn_id"
+    default_conn_name = "google_cloud_bigquery_reservation_default"
+    hook_name = "Google Bigquery Reservation"
+
+    def __init__(
+        self,
+        gcp_conn_id: str = GoogleBaseHook.default_conn_name,
+        location: str | None = None,
+        impersonation_chain: str | Sequence[str] | None = None,
+    ) -> None:
+        super().__init__(
+            gcp_conn_id=gcp_conn_id,
+            impersonation_chain=impersonation_chain,
+        )
+        self.location = location
+        self._client: ReservationServiceClient | None = None
+
+    @staticmethod
+    def _convert_gb_to_kb(value: int) -> int:
+        """
+        Convert GB value to KB.
+
+        :param value: Value to convert
+        """
+        return value * 1073741824
+
+    def get_client(self) -> ReservationServiceClient:
+        """
+        Get reservation service client.
+
+        :return: Google Bigquery Reservation client
+        """
+        if not self._client:
+            self._client = ReservationServiceClient(
+                credentials=self.get_credentials(), client_info=CLIENT_INFO
+            )
+        return self._client
+
+    @GoogleBaseHook.fallback_to_default_project_id
+    def create_bi_reservation(self, project_id: str, size: int) -> None:

Review Comment:
   I try to do like this but currently it is not possible to define `project_id` as an attribute. 
   It seems this is because of `GoogleBaseHook` have defined `project_id` with a decorated @property and it does not have a setter method for `project_id` ([see](https://github.com/apache/airflow/blob/main/airflow/providers/google/common/hooks/base_google.py#L342)).  
   
   I am not confortable to change it in this PR. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pankajastro commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1127687575


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,123 @@
+#

Review Comment:
   +1 for `hooks/bigquery_biengine.py` 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on a diff in pull request #29835: Provider Google Cloud - Add bigquery bi engine reservation operators

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #29835:
URL: https://github.com/apache/airflow/pull/29835#discussion_r1127718456


##########
airflow/providers/google/cloud/hooks/bigquery_reservation.py:
##########
@@ -0,0 +1,123 @@
+#

Review Comment:
   In general we prefer to keep things organized under service and not under API.
   you can see we have bigquery.py, bigtable.py etc..
   It makes it easier to find what you need when everything is under the same service.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org