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 2022/01/27 23:48:06 UTC

[GitHub] [airflow] pateash opened a new pull request #21076: Adding GitHub provider

pateash opened a new pull request #21076:
URL: https://github.com/apache/airflow/pull/21076


   closes: #20300
   
   ---
   ## Description
   
   Adding Github provider based on Python SDK https://github.com/PyGithub/PyGithub
   The SDK has many objects which can be viewed at https://pygithub.readthedocs.io/en/latest/github_objects.html
   
   Users can create their own custom operators leveraging the **PyGithub** SDK, either by leveraging Hook directly or building their operator on GithubOperator by providing **method** and **method_args** from PyGithub  
   results can be further processed by implementing **result_processor** method, please look into **GithubTagSensor** for reference.
   
   
   ```
   github_operator = GithubOperator(
       task_id='github-test',
       github_method="get_repo",
       github_method_args={'full_name_or_id': 'apache/airflow'},
       result_processor=lambda r: r.full_name,
       dag=self.dag,
   )
   ```
   


-- 
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] pateash commented on pull request #21076: Adding GitHub provider

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


   ![image](https://user-images.githubusercontent.com/16856802/151030728-9d5b0b4b-8c4f-41d7-abd7-28606aa5e588.png)
   


-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/.latest-doc-only-change.txt
##########
@@ -0,0 +1,2 @@
+87dc63b65daaf77c4c9f2f6611b72bcc78603d1e
+# TODO: fix this

Review comment:
       You can remove this file.

##########
File path: airflow/providers/github/example_dags/example_github.py
##########
@@ -0,0 +1,57 @@
+# 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.decorators import task
+from airflow.models.dag import DAG
+from airflow.providers.github.hooks.github import GithubHook
+
+
+@task(task_id="github_task")
+def test_github_hook():
+    bucket_name = 'test-influx'

Review comment:
       ```suggestion
       bucket_name = 'test-github'
   ```

##########
File path: airflow/providers/github/example_dags/example_github.py
##########
@@ -0,0 +1,57 @@
+# 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.decorators import task
+from airflow.models.dag import DAG
+from airflow.providers.github.hooks.github import GithubHook
+
+
+@task(task_id="github_task")
+def test_github_hook():
+    bucket_name = 'test-influx'
+    github_hook = GithubHook()
+    client = github_hook.get_conn()
+    print(client)
+    print(f"Organization name {github_hook.org_name}")
+
+    # Make sure enough permissions to create bucket.
+    github_hook.create_bucket(bucket_name, "Bucket to test github connection", github_hook.org_name)
+    github_hook.write(bucket_name, "test_point", "location", "Prague", "temperature", 25.3, True)
+
+    tables = github_hook.query('from(bucket:"test-influx") |> range(start: -10m)')

Review comment:
       Same here.

##########
File path: airflow/providers/github/hooks/github.py
##########
@@ -0,0 +1,77 @@
+#
+# 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 allows to connect to a Github."""
+
+from typing import Dict
+
+from github import Github as GithubClient, PaginatedList
+
+from airflow.hooks.base import BaseHook
+
+
+class GithubHook(BaseHook):
+    """
+    Interact with Github.
+
+    Performs a connection to Github and retrieves client.
+
+    :param github_conn_id: Reference to :ref:`Github connection id <howto/connection:github>`.
+    :type github_conn_id: str
+    """
+
+    conn_name_attr = 'github_conn_id'
+    default_conn_name = 'github_default'
+    conn_type = 'github'
+    hook_name = 'Github'
+
+    def __init__(self, github_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+        self.connection = None
+        self.github_conn_id = github_conn_id
+        self.client = None
+        self.get_conn()
+
+    def get_conn(self) -> GithubClient:
+        """
+        Function that initiates a new Github connection
+        with token and hostname name
+        """
+        if self.client is not None:
+            return self.client
+
+        self.connection = self.get_connection(self.github_conn_id)
+        access_token = self.connection.password
+
+        self.client = GithubClient(login_or_token=access_token)
+        return self.client
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'login', 'extra'],
+            "relabeling": {
+                # 'host': 'Github Enterprise Url',

Review comment:
       How about the commented hosts ?

##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,153 @@
+#
+# 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 typing import Any, Callable, Dict, Optional
+
+from github import Repository, GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Dict) -> Any:
+        return self.github_operator.execute(context=context)
+
+
+class GithubRepositorySensor(GithubSensor):

Review comment:
       ```suggestion
   class BaseGithubRepositorySensor(GithubSensor):
   ```

##########
File path: airflow/providers/github/sensors/github_old.py
##########
@@ -0,0 +1,68 @@
+# #

Review comment:
       Shoudl be removed ?

##########
File path: docs/apache-airflow-providers-github/commits.rst
##########
@@ -0,0 +1,39 @@
+

Review comment:
       This one can be removed or left empty - it should be updated after merge - during release, otherwise it will have wrong commit id (but maybe it is needed to stop test from failing in which case it can remain as it is).

##########
File path: docs/apache-airflow-providers-github/index.rst
##########
@@ -0,0 +1,103 @@
+
+ .. 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.
+
+``apache-airflow-providers-influxdb``
+=======================================
+
+Content
+-------
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Guides
+
+    Connection types <connections/influxdb>
+    Operators <operators/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: References
+
+    Python API <_api/airflow/providers/influxdb/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/influxdb/example_dags>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+    Installing from sources <installing-providers-from-sources>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Commits
+
+    Detailed list of commits <commits>
+
+.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
+
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Commits
+
+    Detailed list of commits <commits>
+
+
+Package apache-airflow-providers-influxdb
+------------------------------------------------------
+
+`InfluxDB <https://www.influxdata.com/>`__

Review comment:
       Few references to influx here.

##########
File path: docs/apache-airflow-providers-github/index.rst
##########
@@ -0,0 +1,103 @@
+
+ .. 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.
+
+``apache-airflow-providers-influxdb``
+=======================================
+
+Content
+-------
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Guides
+
+    Connection types <connections/influxdb>
+    Operators <operators/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: References
+
+    Python API <_api/airflow/providers/influxdb/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/influxdb/example_dags>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+    Installing from sources <installing-providers-from-sources>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Commits
+
+    Detailed list of commits <commits>
+
+.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
+

Review comment:
       You can leave it empty. after that comment. It will be automatically generated when we release.




-- 
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] pateash commented on a change in pull request #21076: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       In Essence, these are identical
   
   ```
   tag_sensor = GithubTagSensor(
       task_id='tag_sensor',
       tag_name='v1.0',
       repository_name="apache/airflow",
       timeout=60,
       poke_interval=10,
       dag=dag,
   )
   
   tag_sensor2 = GithubSensor(
       task_id='tag_sensor2',
       method_name="get_repo",
       method_params={'full_name_or_id': "apache/airflow"},
       result_processor=lambda repo: tag_checker(repo, 'v1.0'),
       timeout=60,
       poke_interval=10,
       dag=dag,
   )
   ```




-- 
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 #21076: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       got it.
   Maybe it would be good to explain about it in the docs as this is quite confusing.




-- 
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] pateash closed pull request #21076: Adding GitHub provider

Posted by GitBox <gi...@apache.org>.
pateash closed pull request #21076:
URL: https://github.com/apache/airflow/pull/21076


   


-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/example_dags/example_github.py
##########
@@ -0,0 +1,57 @@
+# 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.decorators import task
+from airflow.models.dag import DAG
+from airflow.providers.github.hooks.github import GithubHook
+
+
+@task(task_id="github_task")
+def test_github_hook():
+    bucket_name = 'test-influx'

Review comment:
       ```suggestion
       bucket_name = 'test-github'
   ```




-- 
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] pateash commented on pull request #21076: Adding GitHub provider

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


   Hi @potiuk,
   thanks for the comments, Please let me know if anything  else needed.


-- 
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 #21076: Adding GitHub provider

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


   Just in time for the January wave of Providers :)


-- 
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] pateash commented on a change in pull request #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/sensors/github_old.py
##########
@@ -0,0 +1,68 @@
+# #

Review comment:
       yes, of course PR is  WIP. 




-- 
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 merged pull request #21076: Adding GitHub provider

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


   


-- 
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] pateash commented on a change in pull request #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/hooks/github.py
##########
@@ -0,0 +1,77 @@
+#
+# 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 allows to connect to a Github."""
+
+from typing import Dict
+
+from github import Github as GithubClient, PaginatedList
+
+from airflow.hooks.base import BaseHook
+
+
+class GithubHook(BaseHook):
+    """
+    Interact with Github.
+
+    Performs a connection to Github and retrieves client.
+
+    :param github_conn_id: Reference to :ref:`Github connection id <howto/connection:github>`.
+    :type github_conn_id: str
+    """
+
+    conn_name_attr = 'github_conn_id'
+    default_conn_name = 'github_default'
+    conn_type = 'github'
+    hook_name = 'Github'
+
+    def __init__(self, github_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+        self.connection = None
+        self.github_conn_id = github_conn_id
+        self.client = None
+        self.get_conn()
+
+    def get_conn(self) -> GithubClient:
+        """
+        Function that initiates a new Github connection
+        with token and hostname name
+        """
+        if self.client is not None:
+            return self.client
+
+        self.connection = self.get_connection(self.github_conn_id)
+        access_token = self.connection.password
+
+        self.client = GithubClient(login_or_token=access_token)
+        return self.client
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'login', 'extra'],
+            "relabeling": {
+                # 'host': 'Github Enterprise Url',

Review comment:
       for Github Enterprise we have to provide host url,
   I am thinking of adding Github Enterprise support in next release with new connection for GithubEnterprise, 
   What do you think?

##########
File path: airflow/providers/github/sensors/github_old.py
##########
@@ -0,0 +1,68 @@
+# #

Review comment:
       yes, of course PR is  WIP. 

##########
File path: airflow/providers/github/hooks/github.py
##########
@@ -0,0 +1,77 @@
+#
+# 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 allows to connect to a Github."""
+
+from typing import Dict
+
+from github import Github as GithubClient, PaginatedList
+
+from airflow.hooks.base import BaseHook
+
+
+class GithubHook(BaseHook):
+    """
+    Interact with Github.
+
+    Performs a connection to Github and retrieves client.
+
+    :param github_conn_id: Reference to :ref:`Github connection id <howto/connection:github>`.
+    :type github_conn_id: str
+    """
+
+    conn_name_attr = 'github_conn_id'
+    default_conn_name = 'github_default'
+    conn_type = 'github'
+    hook_name = 'Github'
+
+    def __init__(self, github_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+        self.connection = None
+        self.github_conn_id = github_conn_id
+        self.client = None
+        self.get_conn()
+
+    def get_conn(self) -> GithubClient:
+        """
+        Function that initiates a new Github connection
+        with token and hostname name
+        """
+        if self.client is not None:
+            return self.client
+
+        self.connection = self.get_connection(self.github_conn_id)
+        access_token = self.connection.password
+
+        self.client = GithubClient(login_or_token=access_token)
+        return self.client
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'login', 'extra'],
+            "relabeling": {
+                # 'host': 'Github Enterprise Url',

Review comment:
       for Github Enterprise we have to provide host url,
   I am thinking of adding Github Enterprise support in next release with new connection for GithubEnterprise, 
   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] potiuk commented on a change in pull request #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,153 @@
+#
+# 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 typing import Any, Callable, Dict, Optional
+
+from github import Repository, GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Dict) -> Any:
+        return self.github_operator.execute(context=context)
+
+
+class GithubRepositorySensor(GithubSensor):

Review comment:
       ```suggestion
   class BaseGithubRepositorySensor(GithubSensor):
   ```




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/hooks/github.py
##########
@@ -0,0 +1,77 @@
+#
+# 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 allows to connect to a Github."""
+
+from typing import Dict
+
+from github import Github as GithubClient, PaginatedList
+
+from airflow.hooks.base import BaseHook
+
+
+class GithubHook(BaseHook):
+    """
+    Interact with Github.
+
+    Performs a connection to Github and retrieves client.
+
+    :param github_conn_id: Reference to :ref:`Github connection id <howto/connection:github>`.
+    :type github_conn_id: str
+    """
+
+    conn_name_attr = 'github_conn_id'
+    default_conn_name = 'github_default'
+    conn_type = 'github'
+    hook_name = 'Github'
+
+    def __init__(self, github_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+        self.connection = None
+        self.github_conn_id = github_conn_id
+        self.client = None
+        self.get_conn()
+
+    def get_conn(self) -> GithubClient:
+        """
+        Function that initiates a new Github connection
+        with token and hostname name
+        """
+        if self.client is not None:
+            return self.client
+
+        self.connection = self.get_connection(self.github_conn_id)
+        access_token = self.connection.password
+
+        self.client = GithubClient(login_or_token=access_token)
+        return self.client
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'login', 'extra'],
+            "relabeling": {
+                # 'host': 'Github Enterprise Url',

Review comment:
       How about the commented hosts ?




-- 
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] pateash commented on a change in pull request #21076: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       Something like this, result_processor must return **boolean** for sensor to work.
   
   
   ```
    github_tag_sensor = GithubSensor(
               task_id='github_sensor_test',
               method_name='method_name',
               method_params={},
               result_processor=lambda x: x == 'x',
               timeout=60,
               poke_interval=10,
               dag=self.dag,
           )
   
   ```




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: docs/apache-airflow-providers-github/commits.rst
##########
@@ -0,0 +1,39 @@
+

Review comment:
       This one can be removed or left empty - it should be updated after merge - during release, otherwise it will have wrong commit id (but maybe it is needed to stop test from failing in which case it can remain as it is).




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/sensors/github_old.py
##########
@@ -0,0 +1,68 @@
+# #

Review comment:
       Shoudl be removed ?




-- 
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] pateash commented on pull request #21076: Adding GitHub provider

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


   ![image](https://user-images.githubusercontent.com/16856802/151030556-de03c115-f6b3-44f1-96b8-4cf56022283e.png)
   
   ![image](https://user-images.githubusercontent.com/16856802/151030660-ec72786a-c026-4c23-9e7c-a284a9ba5417.png)


-- 
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] pateash commented on a change in pull request #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/hooks/github.py
##########
@@ -0,0 +1,77 @@
+#
+# 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 allows to connect to a Github."""
+
+from typing import Dict
+
+from github import Github as GithubClient, PaginatedList
+
+from airflow.hooks.base import BaseHook
+
+
+class GithubHook(BaseHook):
+    """
+    Interact with Github.
+
+    Performs a connection to Github and retrieves client.
+
+    :param github_conn_id: Reference to :ref:`Github connection id <howto/connection:github>`.
+    :type github_conn_id: str
+    """
+
+    conn_name_attr = 'github_conn_id'
+    default_conn_name = 'github_default'
+    conn_type = 'github'
+    hook_name = 'Github'
+
+    def __init__(self, github_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+        self.connection = None
+        self.github_conn_id = github_conn_id
+        self.client = None
+        self.get_conn()
+
+    def get_conn(self) -> GithubClient:
+        """
+        Function that initiates a new Github connection
+        with token and hostname name
+        """
+        if self.client is not None:
+            return self.client
+
+        self.connection = self.get_connection(self.github_conn_id)
+        access_token = self.connection.password
+
+        self.client = GithubClient(login_or_token=access_token)
+        return self.client
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'login', 'extra'],
+            "relabeling": {
+                # 'host': 'Github Enterprise Url',

Review comment:
       Done, the host will be optional argument and could be passed if we are creating a Github Enteprise connection.




-- 
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 #21076: Adding GitHub provider

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


   Static tests ate failing. Can you fix 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] pateash commented on a change in pull request #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/hooks/github.py
##########
@@ -0,0 +1,77 @@
+#
+# 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 allows to connect to a Github."""
+
+from typing import Dict
+
+from github import Github as GithubClient, PaginatedList
+
+from airflow.hooks.base import BaseHook
+
+
+class GithubHook(BaseHook):
+    """
+    Interact with Github.
+
+    Performs a connection to Github and retrieves client.
+
+    :param github_conn_id: Reference to :ref:`Github connection id <howto/connection:github>`.
+    :type github_conn_id: str
+    """
+
+    conn_name_attr = 'github_conn_id'
+    default_conn_name = 'github_default'
+    conn_type = 'github'
+    hook_name = 'Github'
+
+    def __init__(self, github_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+        self.connection = None
+        self.github_conn_id = github_conn_id
+        self.client = None
+        self.get_conn()
+
+    def get_conn(self) -> GithubClient:
+        """
+        Function that initiates a new Github connection
+        with token and hostname name
+        """
+        if self.client is not None:
+            return self.client
+
+        self.connection = self.get_connection(self.github_conn_id)
+        access_token = self.connection.password
+
+        self.client = GithubClient(login_or_token=access_token)
+        return self.client
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'login', 'extra'],
+            "relabeling": {
+                # 'host': 'Github Enterprise Url',

Review comment:
       for Github Enterprise we have to provide host url,
   I am thinking of adding Github Enterprise support in next release with new connection for GithubEnterprise, 
   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] eladkal commented on a change in pull request #21076: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       It's a bit not orthodox for `BaseGithubRepositorySensor` to inherit from non base hook
   Could you clarify an example for usage of `GithubSensor` directly by the user?

##########
File path: docs/apache-airflow-providers-influxdb/connections/influxdb.rst
##########
@@ -16,34 +16,21 @@
     specific language governing permissions and limitations

Review comment:
       This is wrong...
   you are altering `influxdb.rst` with Github content.
   the `influxdb.rst` should not be modified by this PR.
   You should create a new file `github.rst` in the correct path




-- 
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] pateash commented on a change in pull request #21076: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       In reality, these are identical
   
   ```
   tag_sensor = GithubTagSensor(
       task_id='tag_sensor',
       tag_name='v1.0',
       repository_name="apache/airflow",
       timeout=60,
       poke_interval=10,
       dag=dag,
   )
   
   tag_sensor2 = GithubSensor(
       task_id='tag_sensor2',
       method_name="get_repo",
       method_params={'full_name_or_id': "apache/airflow"},
       result_processor=lambda repo: tag_checker(repo, 'v1.0'),
       timeout=60,
       poke_interval=10,
       dag=dag,
   )
   ```




-- 
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] pateash commented on a change in pull request #21076: WIP: Adding GitHub provider

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



##########
File path: docs/apache-airflow-providers-influxdb/connections/influxdb.rst
##########
@@ -16,34 +16,21 @@
     specific language governing permissions and limitations

Review comment:
       thanks for pointing it out, It looks like there has been a wrong paste.
   




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: docs/apache-airflow-providers-github/index.rst
##########
@@ -0,0 +1,103 @@
+
+ .. 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.
+
+``apache-airflow-providers-influxdb``
+=======================================
+
+Content
+-------
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Guides
+
+    Connection types <connections/influxdb>
+    Operators <operators/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: References
+
+    Python API <_api/airflow/providers/influxdb/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/influxdb/example_dags>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+    Installing from sources <installing-providers-from-sources>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Commits
+
+    Detailed list of commits <commits>
+
+.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
+
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Commits
+
+    Detailed list of commits <commits>
+
+
+Package apache-airflow-providers-influxdb
+------------------------------------------------------
+
+`InfluxDB <https://www.influxdata.com/>`__

Review comment:
       Few references to influx here.




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/example_dags/example_github.py
##########
@@ -0,0 +1,57 @@
+# 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.decorators import task
+from airflow.models.dag import DAG
+from airflow.providers.github.hooks.github import GithubHook
+
+
+@task(task_id="github_task")
+def test_github_hook():
+    bucket_name = 'test-influx'
+    github_hook = GithubHook()
+    client = github_hook.get_conn()
+    print(client)
+    print(f"Organization name {github_hook.org_name}")
+
+    # Make sure enough permissions to create bucket.
+    github_hook.create_bucket(bucket_name, "Bucket to test github connection", github_hook.org_name)
+    github_hook.write(bucket_name, "test_point", "location", "Prague", "temperature", 25.3, True)
+
+    tables = github_hook.query('from(bucket:"test-influx") |> range(start: -10m)')

Review comment:
       Same here.




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: airflow/providers/github/.latest-doc-only-change.txt
##########
@@ -0,0 +1,2 @@
+87dc63b65daaf77c4c9f2f6611b72bcc78603d1e
+# TODO: fix this

Review comment:
       You can remove this file.




-- 
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 #21076: [WIP]: Add GitHub provider

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



##########
File path: docs/apache-airflow-providers-github/index.rst
##########
@@ -0,0 +1,103 @@
+
+ .. 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.
+
+``apache-airflow-providers-influxdb``
+=======================================
+
+Content
+-------
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Guides
+
+    Connection types <connections/influxdb>
+    Operators <operators/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: References
+
+    Python API <_api/airflow/providers/influxdb/index>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/influxdb/example_dags>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Resources
+
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-influxdb/>
+    Installing from sources <installing-providers-from-sources>
+
+.. toctree::
+    :maxdepth: 1
+    :caption: Commits
+
+    Detailed list of commits <commits>
+
+.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
+

Review comment:
       You can leave it empty. after that comment. It will be automatically generated when we release.




-- 
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] pateash commented on a change in pull request #21076: WIP: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       Sure,
   All the Repository level sensors ( which is very common task ) can be created by inheriting **BaseGithubRepositorySensor**, where user just have to provide **result_processor()** as **get_repo()** is already configured.
   
   In all those cases where the sensor could be something else, ex. up on creation  of a access token or webhook and much more, user can inherit GithubSensor and provide both method as well as result_processor to achieve the task.




-- 
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] pateash commented on a change in pull request #21076: [WIP]: Add GitHub provider

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



##########
File path: docs/apache-airflow-providers-github/commits.rst
##########
@@ -0,0 +1,39 @@
+

Review comment:
       sure, keeping it as is for now.
   we can update this upon release.




-- 
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 #21076: Adding GitHub provider

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



##########
File path: airflow/providers/github/sensors/github.py
##########
@@ -0,0 +1,157 @@
+#
+# 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 typing import Any, Callable, Optional
+
+from github import GithubException
+
+from airflow import AirflowException
+from airflow.providers.github.operators.github import GithubOperator
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class GithubSensor(BaseSensorOperator):
+    """
+    Base GithubSensor which can monitor for any change.
+
+    :param github_conn_id: reference to a pre-defined Github Connection
+    :type github_conn_id: str
+    :param method_name: method name from PyGithub to be executed
+    :type method_name: str
+    :param method_params: parameters for the method method_name
+    :type method_params: dict
+    :param result_processor: function that return boolean and act as a sensor response
+    :type result_processor: function
+    """
+
+    def __init__(
+        self,
+        *,
+        method_name: str,
+        github_conn_id: str = 'github_default',
+        method_params: Optional[dict] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.github_conn_id = github_conn_id
+        self.result_processor = None
+        if result_processor is not None:
+            self.result_processor = result_processor
+        self.method_name = method_name
+        self.method_params = method_params
+        self.github_operator = GithubOperator(
+            task_id=self.task_id,
+            github_conn_id=self.github_conn_id,
+            github_method=self.method_name,
+            github_method_args=self.method_params,
+            result_processor=self.result_processor,
+        )
+
+    def poke(self, context: Context) -> bool:
+        return self.github_operator.execute(context=context)
+
+
+class BaseGithubRepositorySensor(GithubSensor):

Review comment:
       Can you give example of how user can use `GithubSensor` without inheriting? Using it directly?
   




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