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/08/14 20:25:44 UTC

[GitHub] [airflow] pateash opened a new pull request, #25714: Adding ClickHouse Provider

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

   closes: #10893 
   
   ---
   ## Description
   
   Adding ClickHouse provider based on its Python SDK https://clickhouse-driver.readthedocs.io/en/latest/
   
   Users can create their own custom operators leveraging the **ClickHouseHook**  directly 
   or building their operator on **ClickHouseOperator** by providing **result_processor**  method,
   
   ```
   operator = ClickHouseOperator(
       task_id='clickhouse_operator',
       sql="SELECT * FROM gettingstarted.clickstream",
       dag=dag,
       result_processor=lambda cursor: print(cursor)
   )
   ```
   
   The sensor can be implemented by SQL
   
   ```
   sensor = ClickHouseSensor(
       task_id="clickhouse_sensor",
       sql="SELECT * FROM gettingstarted.clickstream where customer_id='customer1'",
       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] raphaelauv commented on a diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
raphaelauv commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r945973414


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   ClickHouseQueryOperator ?
   
   also it could maybe be base on the https://pypi.org/project/apache-airflow-providers-common-sql/ ?



-- 
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 #25714: Adding ClickHouse Provider

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

   ![image](https://user-images.githubusercontent.com/16856802/182430905-18d0aefb-4d35-4d69-941f-4638afb4e990.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] eladkal commented on a diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r951132454


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   @pateash can you elaborate on this point?
   Is there a reason why `ClickHouseHook` should not inherit from `DbApiHook`?



-- 
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] raphaelauv commented on a diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
raphaelauv commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r948144754


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   what is 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] pateash commented on pull request #25714: Adding ClickHouse Provider

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

   ![image](https://user-images.githubusercontent.com/16856802/182431066-c4971ce7-b29d-4f49-88f7-c594b67c9d17.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 diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
pateash commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r977691774


##########
airflow/providers/clickhouse/hooks/clickhouse.py:
##########
@@ -0,0 +1,139 @@
+#
+# 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 connecting to a ClickHouse."""
+from typing import Any, Dict, Optional, Tuple
+
+from clickhouse_driver import Client as ClickHouseClient
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class ClickHouseHook(BaseHook):
+    """
+    Interact with ClickHouse.
+    Performs a connection to ClickHouse and retrieves client.
+
+    :param clickhouse_conn_id: Reference to
+        :ref:`ClickHouse connection id<howto/connection:clickhouse>`.
+    :param database: database for the hook, if not provided schema from Connection will be used (optional).
+    """
+
+    conn_name_attr = 'clickhouse_conn_id'
+    default_conn_name = 'clickhouse_default'
+    conn_type = 'clickhouse'
+    hook_name = 'ClickHouse'
+
+    def __init__(
+        self, clickhouse_conn_id: str = default_conn_name, database: Optional[str] = None, *args, **kwargs
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.clickhouse_conn_id = clickhouse_conn_id
+        self.database = database
+
+        self.client: Optional[ClickHouseClient] = None
+        self.get_conn()
+
+    def get_conn(self) -> ClickHouseClient:
+        """Function that initiates a new ClickHouse connection"""
+        if self.client is not None:
+            return self.client
+
+        conn = self.get_connection(self.clickhouse_conn_id)
+        connection_kwargs = conn.extra_dejson.copy()
+
+        if conn.port:
+            connection_kwargs.update(port=int(conn.port))
+        if conn.login:
+            connection_kwargs.update(user=conn.login)
+        if conn.password:
+            connection_kwargs.update(password=conn.password)
+
+        #  if  database is provided use it or use from schema
+        if self.database:
+            connection_kwargs.update(database=self.database)
+        elif conn.schema:
+            connection_kwargs.update(database=conn.schema)

Review Comment:
   Could you please provide more information?
   
   here, if someone wants to override the default schema ( provided by connection ), they can  pass the database argument.



-- 
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] sachebotarev commented on pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
sachebotarev commented on PR #25714:
URL: https://github.com/apache/airflow/pull/25714#issuecomment-1295754000

   Hi  @pateash 
   
   I'm interested in having airflow support ClickHouse, but I don't want to have such a standard implementation.
   
   1. You just copied Anton Bryzgalov's rather old code https://github.com/bryzgaloff/airflow-clickhouse-plugin
   It seems to me that Anton's approach is outdated
   2. clickhouse-driver supports DB API 2.0 so inheriting from DbApiHook and ClickHouseOperator will provide many out of the box solutions. insert_rows( ) can be overridden with Cursor.executemany or just rise exception.
   4. ClickHouseHook provides very limited comparison functionality, for example with ExasolHook (also an analytical database)
   4. ClickHouseOperator does not provide the same capabilities as operators inherited from the modern SQLExecuteQueryOperator
   


-- 
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 diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
pateash commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r977686463


##########
airflow/providers/clickhouse/sensors/clickhouse.py:
##########
@@ -0,0 +1,71 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Any, Callable, Dict, Optional
+
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+from airflow.sensors.sql import SqlSensor
+
+
+class ClickHouseSensor(SqlSensor):

Review Comment:
   HI @eladkal,
   Sorry for the late response, I was OOO.
   
   the reason I didn't used DBApiHook is because it's intended to be used with databases ( mostly transactional supporting sqlalchemy ),
   
   and Clickhouse being a OLAP, i really think that it will be better not to expose methods like insert_rows(), which uses sqlalchemy.
   rather if someone wants to have this functionality they should use the underlying library ( **clickhouse-driver**  ) and implement their own operator using the hook and connection object.



-- 
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 diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
pateash commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r977686463


##########
airflow/providers/clickhouse/sensors/clickhouse.py:
##########
@@ -0,0 +1,71 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Any, Callable, Dict, Optional
+
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+from airflow.sensors.sql import SqlSensor
+
+
+class ClickHouseSensor(SqlSensor):

Review Comment:
   HI @eladkal,
   Sorry for the late response, I was OOO.
   
   the reason I didn't used DBApiHook is because it's intended to be used with databases ( mostly transactional ),
   
   and Clickhouse being a OLAP, i really think that it will be better not to expose methods like insert_rows(), which uses sqlalchemy.
   rather if someone wants to have this functionality they should use the underlying library ( **clickhouse-driver**  ) and implement their own operator using the hook and connection object.



-- 
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 diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
pateash commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r977689882


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   HI @eladkal, @raphaelauv ,
   Sorry for the late response, I was OOO.
   
   the reason I didn't used DBApiHook is because it's intended to be used with databases ( mostly transactional supporting sqlalchemy ),
   
   and Clickhouse being a distributed OLAP, i really think that it will be better not to expose methods like insert_rows(), which uses sqlalchemy.
   rather if someone wants to have this functionality they should use the underlying library ( clickhouse-driver ) and implement their own operator using the hook and connection object.



-- 
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 #25714: Adding ClickHouse Provider

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

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


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

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

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


[GitHub] [airflow] eladkal commented on a diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r951136511


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):
+    """
+    Executes SQL query in a ClickHouse database
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:ClickHouseOperator`
+
+    :param sql: the SQL query to be executed. Can receive a str representing a
+        SQL statement, or you can provide .sql file having the query
+    :param params: substitution parameters for SELECT queries and data for INSERT queries.
+    :param database: database to query, if not provided schema from Connection will be used (optional)
+    :param result_processor: function to further process the Result from ClickHouse
+    :param clickhouse_conn_id: Reference to
+        :ref:`ClickHouse connection id<howto/connection:clickhouse>`.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+
+    template_ext: Sequence[str] = (".sql",)
+    template_fields_renderers = {"sql": "sql"}
+    ui_color = '#ebcc34'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        clickhouse_conn_id: str = 'clickhouse_default',
+        params: Optional[Dict[str, Any]] = None,
+        database: Optional[str] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.clickhouse_conn_id = clickhouse_conn_id
+        self.sql = sql
+        self.params = params
+        self.database = database
+        self.result_processor = result_processor
+
+    def execute(self, context: 'Context'):
+        hook = ClickHouseHook(clickhouse_conn_id=self.clickhouse_conn_id, database=self.database)
+
+        result = hook.query(sql=self.sql, params=self.params)
+        if self.result_processor:
+            self.result_processor(result)

Review Comment:
   Where is `result_processor` implemented?



##########
airflow/providers/clickhouse/sensors/clickhouse.py:
##########
@@ -0,0 +1,71 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import Any, Callable, Dict, Optional
+
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+from airflow.sensors.sql import SqlSensor
+
+
+class ClickHouseSensor(SqlSensor):

Review Comment:
   I asked it elsewhere but if ClickHouseHook will inhert from DBApiHook then we won't need this custom sensor because SqlSensor will support ClickHouse natively https://github.com/pateash/airflow/blob/f7e2ffe42bb7e957e4e16d0cb65e9541c87bd72b/airflow/providers/common/sql/sensors/sql.py#L80



##########
airflow/providers/clickhouse/hooks/clickhouse.py:
##########
@@ -0,0 +1,139 @@
+#
+# 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 connecting to a ClickHouse."""
+from typing import Any, Dict, Optional, Tuple
+
+from clickhouse_driver import Client as ClickHouseClient
+
+from airflow import AirflowException
+from airflow.hooks.base import BaseHook
+
+
+class ClickHouseHook(BaseHook):
+    """
+    Interact with ClickHouse.
+    Performs a connection to ClickHouse and retrieves client.
+
+    :param clickhouse_conn_id: Reference to
+        :ref:`ClickHouse connection id<howto/connection:clickhouse>`.
+    :param database: database for the hook, if not provided schema from Connection will be used (optional).
+    """
+
+    conn_name_attr = 'clickhouse_conn_id'
+    default_conn_name = 'clickhouse_default'
+    conn_type = 'clickhouse'
+    hook_name = 'ClickHouse'
+
+    def __init__(
+        self, clickhouse_conn_id: str = default_conn_name, database: Optional[str] = None, *args, **kwargs
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.clickhouse_conn_id = clickhouse_conn_id
+        self.database = database
+
+        self.client: Optional[ClickHouseClient] = None
+        self.get_conn()
+
+    def get_conn(self) -> ClickHouseClient:
+        """Function that initiates a new ClickHouse connection"""
+        if self.client is not None:
+            return self.client
+
+        conn = self.get_connection(self.clickhouse_conn_id)
+        connection_kwargs = conn.extra_dejson.copy()
+
+        if conn.port:
+            connection_kwargs.update(port=int(conn.port))
+        if conn.login:
+            connection_kwargs.update(user=conn.login)
+        if conn.password:
+            connection_kwargs.update(password=conn.password)
+
+        #  if  database is provided use it or use from schema
+        if self.database:
+            connection_kwargs.update(database=self.database)
+        elif conn.schema:
+            connection_kwargs.update(database=conn.schema)

Review Comment:
   I think this is a source for confusion? Maybe we should customize the connection in the like we do with other 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] eladkal commented on a diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r951132454


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   @pateash can you elaborate on this point?
   I think `ClickHouseHook` should inherit from `DbApiHook`?
   and `ClickHouseOperator` should inherit from `BaseSQLOperator`
   
   Is there a reason why we shouldn't do it?



##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   @pateash can you elaborate on this point?
   I think `ClickHouseHook` should inherit from `DbApiHook` and `ClickHouseOperator` should inherit from `BaseSQLOperator`
   
   Is there a reason why we shouldn't do it?



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

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

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


[GitHub] [airflow] pateash commented on pull request #25714: Adding ClickHouse Provider

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

   Testing providers with sql and template files(*_tf)
   
   ![image](https://user-images.githubusercontent.com/16856802/182429099-29772ec7-ce94-403f-b0a4-680c3aff5c21.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 diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
pateash commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r977688554


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):
+    """
+    Executes SQL query in a ClickHouse database
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:ClickHouseOperator`
+
+    :param sql: the SQL query to be executed. Can receive a str representing a
+        SQL statement, or you can provide .sql file having the query
+    :param params: substitution parameters for SELECT queries and data for INSERT queries.
+    :param database: database to query, if not provided schema from Connection will be used (optional)
+    :param result_processor: function to further process the Result from ClickHouse
+    :param clickhouse_conn_id: Reference to
+        :ref:`ClickHouse connection id<howto/connection:clickhouse>`.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+
+    template_ext: Sequence[str] = (".sql",)
+    template_fields_renderers = {"sql": "sql"}
+    ui_color = '#ebcc34'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        clickhouse_conn_id: str = 'clickhouse_default',
+        params: Optional[Dict[str, Any]] = None,
+        database: Optional[str] = None,
+        result_processor: Optional[Callable] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.clickhouse_conn_id = clickhouse_conn_id
+        self.sql = sql
+        self.params = params
+        self.database = database
+        self.result_processor = result_processor
+
+    def execute(self, context: 'Context'):
+        hook = ClickHouseHook(clickhouse_conn_id=self.clickhouse_conn_id, database=self.database)
+
+        result = hook.query(sql=self.sql, params=self.params)
+        if self.result_processor:
+            self.result_processor(result)

Review Comment:
   result process is a callback which could be passed by the user to proess the result similar to arangodb provider.



-- 
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 diff in pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
pateash commented on code in PR #25714:
URL: https://github.com/apache/airflow/pull/25714#discussion_r948129240


##########
airflow/providers/clickhouse/operators/clickhouse.py:
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.clickhouse.hooks.clickhouse import ClickHouseHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class ClickHouseOperator(BaseOperator):

Review Comment:
   Its 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] github-actions[bot] closed pull request #25714: Adding ClickHouse Provider

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #25714: Adding ClickHouse Provider
URL: https://github.com/apache/airflow/pull/25714


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