You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/12/13 17:48:12 UTC

[GitHub] [airflow] pierrejeambrun opened a new pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

pierrejeambrun opened a new pull request #20263:
URL: https://github.com/apache/airflow/pull/20263


   <!--
   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 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/
   -->
   
   Hello guys,
   
   As suggested in [this issue](#18641), I tried rewriting the opsgenie alert hook with the official python sdk.
   
   This is my first contribution to Airflow, any guidance or feedback would be appreciated :). Especially on how not to break the backward compatibility. (extra args and kwargs given to the constructor will now break, and the return type of `get_conn` has changed)
   
   Best regards,
   
   solves: #18641
   ---
   **^ 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 change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+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 [UPDATING.md](https://github.com/apache/airflow/blob/main/UPDATING.md).
   


-- 
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] potiuk commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772532401



##########
File path: UPDATING.md
##########
@@ -2089,6 +2089,10 @@ Remove unnecessary parameter ``open`` in PostgresHook function ``copy_expert`` f
 
 Change parameter name from ``visibleTo`` to ``visible_to`` in OpsgenieAlertOperator for pylint compatible
 
+`OpsgenieAlertHook` constructor does not take additional arguments or keyword arguments anymore.
+Changed the return type of `OpsgenieAlertHook.get_conn` to return a `opsgenie_sdk.AlertApi` object instead of a `requests.Session` object.
+Changed the return type of `OpsgenieAlertHook.execute` to return a `opsgenie_sdk.SuccessResponse` object instead of a `Any` type.
+

Review comment:
       Yeah. Moving to Opsgenie changelog at the top is best. Not all of it is generated automatically, only list of issues so you can move the comments to the top of the changelog - and I will add the right version after.




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r773346445



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -2117,6 +2117,7 @@ def summarise_total_vs_bad_and_warnings(total: int, bad: int, warns: List[warnin
     ("SelectableGroups dict interface is deprecated. Use select.", "kombu"),
     ("The module cloudant is now deprecated. The replacement is ibmcloudant.", "cloudant"),
     ("distutils Version classes are deprecated. Use packaging.version instead.", "dask"),
+    ("distutils Version classes are deprecated. Use packaging.version instead.", "fsspec"),

Review comment:
       Yes ofc, done.




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769521116



##########
File path: tests/providers/opsgenie/hooks/test_opsgenie_alert.py
##########
@@ -45,7 +45,7 @@ class TestOpsgenieAlertHook(unittest.TestCase):
             {'id': '80564037-1984-4f38-b98e-8a1f662df552', 'type': 'schedule'},
             {'name': 'First Responders Schedule', 'type': 'schedule'},
         ],
-        'visibleTo': [
+        'visible_to': [

Review comment:
       Indeed, the official sdk take different parameters (python style), so `visibleTo` becomes `visible_to` and use an internal mapping to construct the request. Now it's the same parameter as the corresponding operator.
   
   The API is expecting `visibleTo` but the operator was directly passing to the hook a 'visible_to' parameter, [see here](https://github.com/apache/airflow/blob/866a601b76e219b3c043e1dbbc8fb22300866351/airflow/providers/opsgenie/operators/opsgenie_alert.py#L122). To be honest I am not even sure the operator was working before with this parameter.
   The test for the hook was correct though.
   
   I will still upgrade the major version so it's not ambiguous, what do you thing ?
   
   More info in the [CreateAlertPayload](https://github.com/opsgenie/opsgenie-python-sdk/blob/master/opsgenie_sdk/api/alert/create_alert_payload.py#L50)




-- 
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] pierrejeambrun commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-993371141


   Just amended to fix checks.


-- 
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] pierrejeambrun edited a comment on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun edited a comment on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998712679


   Hi,
   
   I have fixed that, and the job successfully ran locally, we should be good now.
   
   (I added `fsspec` module as known depecrated message due to distutils deprecation of Version)


-- 
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] pierrejeambrun edited a comment on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun edited a comment on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-999492021


   My pleasure, it was really interesting helping on that.
   
   I will try to first close my other PR https://github.com/apache/airflow/pull/20386 that will certainly require more work. (I’ll also be OOO for a week)
   
   After that if it is still available I can take 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] uranusjr commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769475283



##########
File path: tests/providers/opsgenie/hooks/test_opsgenie_alert.py
##########
@@ -45,7 +45,7 @@ class TestOpsgenieAlertHook(unittest.TestCase):
             {'id': '80564037-1984-4f38-b98e-8a1f662df552', 'type': 'schedule'},
             {'name': 'First Responders Schedule', 'type': 'schedule'},
         ],
-        'visibleTo': [
+        'visible_to': [

Review comment:
       So this is a backward incompatible change right? We’ll need to bump the provider’s major version if so.

##########
File path: airflow/providers/opsgenie/example_dags/example_opsgenie_alert.py
##########
@@ -0,0 +1,31 @@
+# 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 datetime import datetime
+
+from airflow import DAG
+from airflow.providers.opsgenie.operators.opsgenie_alert import OpsgenieAlertOperator
+
+with DAG(
+    dag_id="opsgenie_alert_operator_dag",
+    schedule_interval=None,
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+) as dag:
+
+    # [START howto_opsgenie_alert_operator]
+    opsgenie_alert_operator = OpsgenieAlertOperator(task_id="opsgenie_task", message="Hello World!")
+    # [END howto_opsgenie_alert_operator]

Review comment:
       Do you plan to add documentation as well? Because otherwise this START-END combination is not needed (it’s used to load example code into documentation).




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769558873



##########
File path: airflow/providers/opsgenie/example_dags/example_opsgenie_alert.py
##########
@@ -0,0 +1,31 @@
+# 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 datetime import datetime
+
+from airflow import DAG
+from airflow.providers.opsgenie.operators.opsgenie_alert import OpsgenieAlertOperator
+
+with DAG(
+    dag_id="opsgenie_alert_operator_dag",
+    schedule_interval=None,
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+) as dag:
+
+    # [START howto_opsgenie_alert_operator]
+    opsgenie_alert_operator = OpsgenieAlertOperator(task_id="opsgenie_task", message="Hello World!")
+    # [END howto_opsgenie_alert_operator]

Review comment:
       I just added the doc, thank you :)




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772570717



##########
File path: setup.py
##########
@@ -405,6 +405,9 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
 odbc = [
     'pyodbc',
 ]
+opsgenie_sdk = [

Review comment:
       I updated the variable name.




-- 
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 change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772531440



##########
File path: airflow/providers/opsgenie/hooks/opsgenie_alert.py
##########
@@ -46,46 +51,50 @@ class OpsgenieAlertHook(HttpHook):
     conn_type = 'opsgenie'
     hook_name = 'Opsgenie'
 
-    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default', *args, **kwargs) -> None:
-        super().__init__(http_conn_id=opsgenie_conn_id, *args, **kwargs)  # type: ignore[misc]
+    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default') -> None:
+        super().__init__()  # type: ignore[misc]
+        self.conn_id = opsgenie_conn_id
+        configuration = Configuration()
+        conn = self.get_connection(self.conn_id)
+        configuration.api_key['Authorization'] = conn.password
+        configuration.host = conn.host or 'https://api.opsgenie.com'
+        self.alert_api_instance = AlertApi(ApiClient(configuration))
 
     def _get_api_key(self) -> str:
-        """Get Opsgenie api_key for creating alert"""
-        conn = self.get_connection(self.http_conn_id)
-        api_key = conn.password
-        if not api_key:
-            raise AirflowException(
-                'Opsgenie API Key is required for this hook, please check your conn_id configuration.'
-            )
-        return api_key
-
-    def get_conn(self, headers: Optional[dict] = None) -> requests.Session:
         """
-        Overwrite HttpHook get_conn because this hook just needs base_url
-        and headers, and does not need generic params
+        Get the API key from the connection
 
-        :param headers: additional headers to be passed through as a dictionary
-        :type headers: dict
+        :return: API key
+        :rtype: str
         """
-        conn = self.get_connection(self.http_conn_id)
-        self.base_url = conn.host if conn.host else 'https://api.opsgenie.com'
-        session = requests.Session()
-        if headers:
-            session.headers.update(headers)
-        return session
-
-    def execute(self, payload: Optional[dict] = None) -> Any:
+        conn = self.get_connection(self.conn_id)
+        return conn.password
+
+    def get_conn(self) -> AlertApi:
+        """
+        Get the underlying AlertApi client
+
+        :return: AlertApi client
+        :rtype: opsgenie_sdk.AlertApi
+        """
+        return self.alert_api_instance
+
+    def execute(self, payload: Optional[dict] = None) -> SuccessResponse:

Review comment:
       I would also say that this is a very good opportunity to change this function name.
   execute isn't self explanatory and also the sdk itself doesn't have it. If in the future we will add more functions to the hook it would be harder to understand what execute does.
   
   I think this function should be called `create_alert`




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769521116



##########
File path: tests/providers/opsgenie/hooks/test_opsgenie_alert.py
##########
@@ -45,7 +45,7 @@ class TestOpsgenieAlertHook(unittest.TestCase):
             {'id': '80564037-1984-4f38-b98e-8a1f662df552', 'type': 'schedule'},
             {'name': 'First Responders Schedule', 'type': 'schedule'},
         ],
-        'visibleTo': [
+        'visible_to': [

Review comment:
       Indeed, the official sdk take different parameters (python style), so `visibleTo` becomes `visible_to` and use an internal mapping to construct the request. More info in the [CreateAlertPayload](https://github.com/opsgenie/opsgenie-python-sdk/blob/master/opsgenie_sdk/api/alert/create_alert_payload.py#L50).
   
   Now the `OpsgenieAlertOperator` takes the same parameter as `OpsgenieAlertHook.execute` it's the same parameter as the corresponding operator.
   
   The API is expecting `visibleTo` but the operator was directly passing to the hook a 'visible_to' parameter, [see here](https://github.com/apache/airflow/blob/866a601b76e219b3c043e1dbbc8fb22300866351/airflow/providers/opsgenie/operators/opsgenie_alert.py#L122). To be honest I am not even sure the operator was working before with this parameter.
   The test for the hook was correct though.
   
   I will still upgrade the major version so it's not ambiguous, what do you think ?




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769521116



##########
File path: tests/providers/opsgenie/hooks/test_opsgenie_alert.py
##########
@@ -45,7 +45,7 @@ class TestOpsgenieAlertHook(unittest.TestCase):
             {'id': '80564037-1984-4f38-b98e-8a1f662df552', 'type': 'schedule'},
             {'name': 'First Responders Schedule', 'type': 'schedule'},
         ],
-        'visibleTo': [
+        'visible_to': [

Review comment:
       Indeed, the official sdk take different parameters (python style), so `visibleTo` becomes `visible_to` and use an internal mapping to construct the request. More info in the [CreateAlertPayload](https://github.com/opsgenie/opsgenie-python-sdk/blob/master/opsgenie_sdk/api/alert/create_alert_payload.py#L50).
   
   Now the `OpsgenieAlertOperator` takes the same parameter as `OpsgenieAlertHook.execute` it's the same parameter as the corresponding operator.
   
   The API is expecting `visibleTo` but the operator was directly passing to the hook a 'visible_to' parameter, [see here](https://github.com/apache/airflow/blob/866a601b76e219b3c043e1dbbc8fb22300866351/airflow/providers/opsgenie/operators/opsgenie_alert.py#L122). To be honest I am not even sure the operator was working before with this parameter.
   The test for the hook was correct though.
   
   I agree we should upgrade the major so this is not ambiguous.




-- 
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] boring-cyborg[bot] commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-999054441


   Awesome work, congrats on your first merged pull request!
   


-- 
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] pierrejeambrun commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998712679


   Hi,
   
   I have fixed that, and the job successfully ran locally, we should be good now.


-- 
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] pierrejeambrun commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-999492021


   My pleasure, it was really interesting helping on that.
   
   I will try to first close my other PR https://github.com/apache/airflow/pull/20386 that will certainly require more work. (I’ll also be OOO for a week)
   
   After that I think if it is still available I can take 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 pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-999500628


   @pierrejeambrun you can comment on the issue and i'll assign you


-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769521116



##########
File path: tests/providers/opsgenie/hooks/test_opsgenie_alert.py
##########
@@ -45,7 +45,7 @@ class TestOpsgenieAlertHook(unittest.TestCase):
             {'id': '80564037-1984-4f38-b98e-8a1f662df552', 'type': 'schedule'},
             {'name': 'First Responders Schedule', 'type': 'schedule'},
         ],
-        'visibleTo': [
+        'visible_to': [

Review comment:
       Indeed, the official sdk take different parameters (python style), so `visibleTo` becomes `visible_to` and use an internal mapping to construct the request. More info in the [CreateAlertPayload](https://github.com/opsgenie/opsgenie-python-sdk/blob/master/opsgenie_sdk/api/alert/create_alert_payload.py#L50).
   
   Now the `OpsgenieAlertOperator` takes the same parameter as `OpsgenieAlertHook.execute`.
   
   The API is expecting `visibleTo` but the operator was directly passing to the hook a 'visible_to' parameter, [see here](https://github.com/apache/airflow/blob/866a601b76e219b3c043e1dbbc8fb22300866351/airflow/providers/opsgenie/operators/opsgenie_alert.py#L122). To be honest I am not even sure the operator was working before with this parameter.
   The test for the hook was correct though.
   
   I agree we should upgrade the major so this is not ambiguous.




-- 
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] potiuk commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772532804



##########
File path: airflow/providers/opsgenie/hooks/opsgenie_alert.py
##########
@@ -46,46 +51,50 @@ class OpsgenieAlertHook(HttpHook):
     conn_type = 'opsgenie'
     hook_name = 'Opsgenie'
 
-    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default', *args, **kwargs) -> None:
-        super().__init__(http_conn_id=opsgenie_conn_id, *args, **kwargs)  # type: ignore[misc]
+    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default') -> None:
+        super().__init__()  # type: ignore[misc]
+        self.conn_id = opsgenie_conn_id
+        configuration = Configuration()
+        conn = self.get_connection(self.conn_id)
+        configuration.api_key['Authorization'] = conn.password
+        configuration.host = conn.host or 'https://api.opsgenie.com'
+        self.alert_api_instance = AlertApi(ApiClient(configuration))
 
     def _get_api_key(self) -> str:
-        """Get Opsgenie api_key for creating alert"""
-        conn = self.get_connection(self.http_conn_id)
-        api_key = conn.password
-        if not api_key:
-            raise AirflowException(
-                'Opsgenie API Key is required for this hook, please check your conn_id configuration.'
-            )
-        return api_key
-
-    def get_conn(self, headers: Optional[dict] = None) -> requests.Session:
         """
-        Overwrite HttpHook get_conn because this hook just needs base_url
-        and headers, and does not need generic params
+        Get the API key from the connection
 
-        :param headers: additional headers to be passed through as a dictionary
-        :type headers: dict
+        :return: API key
+        :rtype: str
         """
-        conn = self.get_connection(self.http_conn_id)
-        self.base_url = conn.host if conn.host else 'https://api.opsgenie.com'
-        session = requests.Session()
-        if headers:
-            session.headers.update(headers)
-        return session
-
-    def execute(self, payload: Optional[dict] = None) -> Any:
+        conn = self.get_connection(self.conn_id)
+        return conn.password
+
+    def get_conn(self) -> AlertApi:
+        """
+        Get the underlying AlertApi client
+
+        :return: AlertApi client
+        :rtype: opsgenie_sdk.AlertApi
+        """
+        return self.alert_api_instance
+
+    def execute(self, payload: Optional[dict] = None) -> SuccessResponse:

Review comment:
       Agree with Elad!




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772571077



##########
File path: airflow/providers/opsgenie/hooks/opsgenie_alert.py
##########
@@ -46,46 +51,50 @@ class OpsgenieAlertHook(HttpHook):
     conn_type = 'opsgenie'
     hook_name = 'Opsgenie'
 
-    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default', *args, **kwargs) -> None:
-        super().__init__(http_conn_id=opsgenie_conn_id, *args, **kwargs)  # type: ignore[misc]
+    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default') -> None:
+        super().__init__()  # type: ignore[misc]
+        self.conn_id = opsgenie_conn_id
+        configuration = Configuration()
+        conn = self.get_connection(self.conn_id)
+        configuration.api_key['Authorization'] = conn.password
+        configuration.host = conn.host or 'https://api.opsgenie.com'
+        self.alert_api_instance = AlertApi(ApiClient(configuration))
 
     def _get_api_key(self) -> str:
-        """Get Opsgenie api_key for creating alert"""
-        conn = self.get_connection(self.http_conn_id)
-        api_key = conn.password
-        if not api_key:
-            raise AirflowException(
-                'Opsgenie API Key is required for this hook, please check your conn_id configuration.'
-            )
-        return api_key
-
-    def get_conn(self, headers: Optional[dict] = None) -> requests.Session:
         """
-        Overwrite HttpHook get_conn because this hook just needs base_url
-        and headers, and does not need generic params
+        Get the API key from the connection
 
-        :param headers: additional headers to be passed through as a dictionary
-        :type headers: dict
+        :return: API key
+        :rtype: str
         """
-        conn = self.get_connection(self.http_conn_id)
-        self.base_url = conn.host if conn.host else 'https://api.opsgenie.com'
-        session = requests.Session()
-        if headers:
-            session.headers.update(headers)
-        return session
-
-    def execute(self, payload: Optional[dict] = None) -> Any:
+        conn = self.get_connection(self.conn_id)
+        return conn.password
+
+    def get_conn(self) -> AlertApi:
+        """
+        Get the underlying AlertApi client
+
+        :return: AlertApi client
+        :rtype: opsgenie_sdk.AlertApi
+        """
+        return self.alert_api_instance
+
+    def execute(self, payload: Optional[dict] = None) -> SuccessResponse:

Review comment:
       You are right, this makes more sense :+1: 




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772571077



##########
File path: airflow/providers/opsgenie/hooks/opsgenie_alert.py
##########
@@ -46,46 +51,50 @@ class OpsgenieAlertHook(HttpHook):
     conn_type = 'opsgenie'
     hook_name = 'Opsgenie'
 
-    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default', *args, **kwargs) -> None:
-        super().__init__(http_conn_id=opsgenie_conn_id, *args, **kwargs)  # type: ignore[misc]
+    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default') -> None:
+        super().__init__()  # type: ignore[misc]
+        self.conn_id = opsgenie_conn_id
+        configuration = Configuration()
+        conn = self.get_connection(self.conn_id)
+        configuration.api_key['Authorization'] = conn.password
+        configuration.host = conn.host or 'https://api.opsgenie.com'
+        self.alert_api_instance = AlertApi(ApiClient(configuration))
 
     def _get_api_key(self) -> str:
-        """Get Opsgenie api_key for creating alert"""
-        conn = self.get_connection(self.http_conn_id)
-        api_key = conn.password
-        if not api_key:
-            raise AirflowException(
-                'Opsgenie API Key is required for this hook, please check your conn_id configuration.'
-            )
-        return api_key
-
-    def get_conn(self, headers: Optional[dict] = None) -> requests.Session:
         """
-        Overwrite HttpHook get_conn because this hook just needs base_url
-        and headers, and does not need generic params
+        Get the API key from the connection
 
-        :param headers: additional headers to be passed through as a dictionary
-        :type headers: dict
+        :return: API key
+        :rtype: str
         """
-        conn = self.get_connection(self.http_conn_id)
-        self.base_url = conn.host if conn.host else 'https://api.opsgenie.com'
-        session = requests.Session()
-        if headers:
-            session.headers.update(headers)
-        return session
-
-    def execute(self, payload: Optional[dict] = None) -> Any:
+        conn = self.get_connection(self.conn_id)
+        return conn.password
+
+    def get_conn(self) -> AlertApi:
+        """
+        Get the underlying AlertApi client
+
+        :return: AlertApi client
+        :rtype: opsgenie_sdk.AlertApi
+        """
+        return self.alert_api_instance
+
+    def execute(self, payload: Optional[dict] = None) -> SuccessResponse:

Review comment:
       You are right, this makes more sense.




-- 
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] boring-cyborg[bot] commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-992720691


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
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] pierrejeambrun edited a comment on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun edited a comment on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-997857171


   @potiuk, thanks for that, I just installed them and fixed the `provider-yamls` pre-commit. Do you want me to push with the --force-with-lease flag?


-- 
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] pierrejeambrun edited a comment on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun edited a comment on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-997857171


   @potiuk, thanks for that, I just installed them and fixed the `provider-yamls` pre-commit.


-- 
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 pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-999110551


   Thanks @pierrejeambrun for your contribution!
   If you are interested in taking another task check https://github.com/apache/airflow/issues/8937 which is kinda similar


-- 
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 pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998566839


   There is a still a failure
   https://github.com/apache/airflow/runs/4591740391?check_suite_focus=true#step:6:865
   
   You probably need to add a fix similar to https://github.com/apache/airflow/pull/20420/files
   See the instructions in the test failure message


-- 
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] pierrejeambrun edited a comment on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun edited a comment on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998712679


   Hi,
   
   I have fixed that, and the job successfully ran locally, we should be good now.
   
   (I added `fsspec` module as known depecrated message due to distutils deprecation of `Version`)


-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r769521116



##########
File path: tests/providers/opsgenie/hooks/test_opsgenie_alert.py
##########
@@ -45,7 +45,7 @@ class TestOpsgenieAlertHook(unittest.TestCase):
             {'id': '80564037-1984-4f38-b98e-8a1f662df552', 'type': 'schedule'},
             {'name': 'First Responders Schedule', 'type': 'schedule'},
         ],
-        'visibleTo': [
+        'visible_to': [

Review comment:
       Indeed, the official sdk take different parameters (python style), so `visibleTo` becomes `visible_to` and use an internal mapping to construct the request. Now it's the same parameter as the corresponding operator.
   
   The API is expecting `visibleTo` but the operator was directly passing to the hook a 'visible_to' parameter, [see here](https://github.com/apache/airflow/blob/866a601b76e219b3c043e1dbbc8fb22300866351/airflow/providers/opsgenie/operators/opsgenie_alert.py#L122). To be honest I am not even sure the operator was working before with this parameter.
   The test for the hook was correct though.
   
   I will still upgrade the major version so it's not ambiguous, what do you think ?
   
   More info in the [CreateAlertPayload](https://github.com/opsgenie/opsgenie-python-sdk/blob/master/opsgenie_sdk/api/alert/create_alert_payload.py#L50)




-- 
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] potiuk commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-997235842


   Static checks failing (I heartily recommend pre-commit installation)


-- 
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 #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-997233593


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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 change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772522154



##########
File path: setup.py
##########
@@ -405,6 +405,9 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
 odbc = [
     'pyodbc',
 ]
+opsgenie_sdk = [

Review comment:
       do we need the `_sdk`?

##########
File path: UPDATING.md
##########
@@ -2089,6 +2089,10 @@ Remove unnecessary parameter ``open`` in PostgresHook function ``copy_expert`` f
 
 Change parameter name from ``visibleTo`` to ``visible_to`` in OpsgenieAlertOperator for pylint compatible
 
+`OpsgenieAlertHook` constructor does not take additional arguments or keyword arguments anymore.
+Changed the return type of `OpsgenieAlertHook.get_conn` to return a `opsgenie_sdk.AlertApi` object instead of a `requests.Session` object.
+Changed the return type of `OpsgenieAlertHook.execute` to return a `opsgenie_sdk.SuccessResponse` object instead of a `Any` type.
+

Review comment:
       This section is Updating of Airflow 2.0 - this is not relevant here.
   Users who switch to 2.0 has no action item as the provider we are now upgrading is not compatible with Airflow 2.
   
   These entries means that this PR is a breaking change (which we welcome!)
   @potiuk since you generate the change log automatically I wonder what is the best way to notify the users?
   Should we ask to move these records to 
   https://github.com/apache/airflow/blob/main/airflow/providers/opsgenie/CHANGELOG.rst
   with 3.0.0 ?
   Alternatively we can maybe put this information in the PR description or/and in the commit message and the change log would refer to 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] pierrejeambrun commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998989783


   @eladkal We should be good :smile: 


-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772571077



##########
File path: airflow/providers/opsgenie/hooks/opsgenie_alert.py
##########
@@ -46,46 +51,50 @@ class OpsgenieAlertHook(HttpHook):
     conn_type = 'opsgenie'
     hook_name = 'Opsgenie'
 
-    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default', *args, **kwargs) -> None:
-        super().__init__(http_conn_id=opsgenie_conn_id, *args, **kwargs)  # type: ignore[misc]
+    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default') -> None:
+        super().__init__()  # type: ignore[misc]
+        self.conn_id = opsgenie_conn_id
+        configuration = Configuration()
+        conn = self.get_connection(self.conn_id)
+        configuration.api_key['Authorization'] = conn.password
+        configuration.host = conn.host or 'https://api.opsgenie.com'
+        self.alert_api_instance = AlertApi(ApiClient(configuration))
 
     def _get_api_key(self) -> str:
-        """Get Opsgenie api_key for creating alert"""
-        conn = self.get_connection(self.http_conn_id)
-        api_key = conn.password
-        if not api_key:
-            raise AirflowException(
-                'Opsgenie API Key is required for this hook, please check your conn_id configuration.'
-            )
-        return api_key
-
-    def get_conn(self, headers: Optional[dict] = None) -> requests.Session:
         """
-        Overwrite HttpHook get_conn because this hook just needs base_url
-        and headers, and does not need generic params
+        Get the API key from the connection
 
-        :param headers: additional headers to be passed through as a dictionary
-        :type headers: dict
+        :return: API key
+        :rtype: str
         """
-        conn = self.get_connection(self.http_conn_id)
-        self.base_url = conn.host if conn.host else 'https://api.opsgenie.com'
-        session = requests.Session()
-        if headers:
-            session.headers.update(headers)
-        return session
-
-    def execute(self, payload: Optional[dict] = None) -> Any:
+        conn = self.get_connection(self.conn_id)
+        return conn.password
+
+    def get_conn(self) -> AlertApi:
+        """
+        Get the underlying AlertApi client
+
+        :return: AlertApi client
+        :rtype: opsgenie_sdk.AlertApi
+        """
+        return self.alert_api_instance
+
+    def execute(self, payload: Optional[dict] = None) -> SuccessResponse:

Review comment:
       You are right, this makes more sense :)




-- 
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 merged pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal merged pull request #20263:
URL: https://github.com/apache/airflow/pull/20263


   


-- 
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] pierrejeambrun commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-997857171


   @potiuk, thanks for that, I just installed them and fixed the `provider-yamls` pre-commit.


-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772575373



##########
File path: UPDATING.md
##########
@@ -2089,6 +2089,10 @@ Remove unnecessary parameter ``open`` in PostgresHook function ``copy_expert`` f
 
 Change parameter name from ``visibleTo`` to ``visible_to`` in OpsgenieAlertOperator for pylint compatible
 
+`OpsgenieAlertHook` constructor does not take additional arguments or keyword arguments anymore.
+Changed the return type of `OpsgenieAlertHook.get_conn` to return a `opsgenie_sdk.AlertApi` object instead of a `requests.Session` object.
+Changed the return type of `OpsgenieAlertHook.execute` to return a `opsgenie_sdk.SuccessResponse` object instead of a `Any` type.
+

Review comment:
       As requested I moved this section to the top of `/airflow/providers/opsgenie/CHANGELOG.rst`. I have put that in a [breaking change section](https://github.com/apache/airflow/pull/20263/files#diff-58c14e916c12a35f7fa03cd5e28ce661151c914c0786dfac7c3c0a1a8b96aa69R22) for now.
   
   @potiuk  let me know if this looks good to you. 




-- 
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] pierrejeambrun commented on a change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r773346445



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -2117,6 +2117,7 @@ def summarise_total_vs_bad_and_warnings(total: int, bad: int, warns: List[warnin
     ("SelectableGroups dict interface is deprecated. Use select.", "kombu"),
     ("The module cloudant is now deprecated. The replacement is ibmcloudant.", "cloudant"),
     ("distutils Version classes are deprecated. Use packaging.version instead.", "dask"),
+    ("distutils Version classes are deprecated. Use packaging.version instead.", "fsspec"),

Review comment:
       Yes ofc




-- 
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 change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r772531440



##########
File path: airflow/providers/opsgenie/hooks/opsgenie_alert.py
##########
@@ -46,46 +51,50 @@ class OpsgenieAlertHook(HttpHook):
     conn_type = 'opsgenie'
     hook_name = 'Opsgenie'
 
-    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default', *args, **kwargs) -> None:
-        super().__init__(http_conn_id=opsgenie_conn_id, *args, **kwargs)  # type: ignore[misc]
+    def __init__(self, opsgenie_conn_id: str = 'opsgenie_default') -> None:
+        super().__init__()  # type: ignore[misc]
+        self.conn_id = opsgenie_conn_id
+        configuration = Configuration()
+        conn = self.get_connection(self.conn_id)
+        configuration.api_key['Authorization'] = conn.password
+        configuration.host = conn.host or 'https://api.opsgenie.com'
+        self.alert_api_instance = AlertApi(ApiClient(configuration))
 
     def _get_api_key(self) -> str:
-        """Get Opsgenie api_key for creating alert"""
-        conn = self.get_connection(self.http_conn_id)
-        api_key = conn.password
-        if not api_key:
-            raise AirflowException(
-                'Opsgenie API Key is required for this hook, please check your conn_id configuration.'
-            )
-        return api_key
-
-    def get_conn(self, headers: Optional[dict] = None) -> requests.Session:
         """
-        Overwrite HttpHook get_conn because this hook just needs base_url
-        and headers, and does not need generic params
+        Get the API key from the connection
 
-        :param headers: additional headers to be passed through as a dictionary
-        :type headers: dict
+        :return: API key
+        :rtype: str
         """
-        conn = self.get_connection(self.http_conn_id)
-        self.base_url = conn.host if conn.host else 'https://api.opsgenie.com'
-        session = requests.Session()
-        if headers:
-            session.headers.update(headers)
-        return session
-
-    def execute(self, payload: Optional[dict] = None) -> Any:
+        conn = self.get_connection(self.conn_id)
+        return conn.password
+
+    def get_conn(self) -> AlertApi:
+        """
+        Get the underlying AlertApi client
+
+        :return: AlertApi client
+        :rtype: opsgenie_sdk.AlertApi
+        """
+        return self.alert_api_instance
+
+    def execute(self, payload: Optional[dict] = None) -> SuccessResponse:

Review comment:
       I would also say that this is a very good opportunity to change this function name.
   execute isn't self explanatory and also the sdk itself doesn't have it. If in the future we will add more functions to the hook it would be harder to understand what execute does.
   
   I think this function should be called `create_alert`
   We may add in the future also (`assign_alert`, `close_alert` and many others)




-- 
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] pierrejeambrun commented on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun commented on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998261343


   Yes of course, I juste rebased onto the latest main version.


-- 
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] pierrejeambrun edited a comment on pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
pierrejeambrun edited a comment on pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#issuecomment-998712679


   Hi,
   
   I have fixed that, and the job successfully ran locally, we should be good now.
   
   We were missing warnings from `fsspec`.


-- 
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 change in pull request #20263: rewrite opsgenie alert hook with official python sdk, related issue #18641

Posted by GitBox <gi...@apache.org>.
eladkal commented on a change in pull request #20263:
URL: https://github.com/apache/airflow/pull/20263#discussion_r773327411



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -2117,6 +2117,7 @@ def summarise_total_vs_bad_and_warnings(total: int, bad: int, warns: List[warnin
     ("SelectableGroups dict interface is deprecated. Use select.", "kombu"),
     ("The module cloudant is now deprecated. The replacement is ibmcloudant.", "cloudant"),
     ("distutils Version classes are deprecated. Use packaging.version instead.", "dask"),
+    ("distutils Version classes are deprecated. Use packaging.version instead.", "fsspec"),

Review comment:
       In the mean time https://github.com/apache/airflow/pull/20444 was merged
   so can you please remove this change and rebase?




-- 
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