Skip to content

Commit

Permalink
Add ImpalaHook (#26970)
Browse files Browse the repository at this point in the history
Add `ImpalaHook`

Co-authored-by: eladkal <45845474+eladkal@users.noreply.github.com>
  • Loading branch information
Hossein Torabi and eladkal committed Jan 6, 2023
1 parent dcfc604 commit 6d09fc7
Show file tree
Hide file tree
Showing 28 changed files with 633 additions and 200 deletions.
1 change: 1 addition & 0 deletions .github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml
Expand Up @@ -33,6 +33,7 @@ body:
- apache-flink
- apache-hdfs
- apache-hive
- apache-impala
- apache-kylin
- apache-livy
- apache-pig
Expand Down
4 changes: 2 additions & 2 deletions CONTRIBUTING.rst
Expand Up @@ -611,8 +611,8 @@ This is the full list of those extras:

.. START EXTRAS HERE
airbyte, alibaba, all, all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.drill,
apache.druid, apache.flink, apache.hdfs, apache.hive, apache.kylin, apache.livy, apache.pig,
apache.pinot, apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas,
apache.druid, apache.flink, apache.hdfs, apache.hive, apache.impala, apache.kylin, apache.livy,
apache.pig, apache.pinot, apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas,
atlassian.jira, aws, azure, cassandra, celery, cgroups, cloudant, cncf.kubernetes, common.sql,
crypto, dask, databricks, datadog, dbt.cloud, deprecated_api, devel, devel_all, devel_ci,
devel_hadoop, dingding, discord, doc, doc_gen, docker, druid, elasticsearch, exasol, facebook, ftp,
Expand Down
4 changes: 2 additions & 2 deletions INSTALL
Expand Up @@ -95,8 +95,8 @@ The list of available extras:

# START EXTRAS HERE
airbyte, alibaba, all, all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.drill,
apache.druid, apache.flink, apache.hdfs, apache.hive, apache.kylin, apache.livy, apache.pig,
apache.pinot, apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas,
apache.druid, apache.flink, apache.hdfs, apache.hive, apache.impala, apache.kylin, apache.livy,
apache.pig, apache.pinot, apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas,
atlassian.jira, aws, azure, cassandra, celery, cgroups, cloudant, cncf.kubernetes, common.sql,
crypto, dask, databricks, datadog, dbt.cloud, deprecated_api, devel, devel_all, devel_ci,
devel_hadoop, dingding, discord, doc, doc_gen, docker, druid, elasticsearch, exasol, facebook, ftp,
Expand Down
25 changes: 25 additions & 0 deletions airflow/providers/apache/impala/CHANGELOG.rst
@@ -0,0 +1,25 @@
.. 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.
Changelog
---------

1.0.0
.....

Initial version of the provider.
17 changes: 17 additions & 0 deletions airflow/providers/apache/impala/__init__.py
@@ -0,0 +1,17 @@
#
# 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.
17 changes: 17 additions & 0 deletions airflow/providers/apache/impala/hooks/__init__.py
@@ -0,0 +1,17 @@
#
# 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.
42 changes: 42 additions & 0 deletions airflow/providers/apache/impala/hooks/impala.py
@@ -0,0 +1,42 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import annotations

from impala.dbapi import connect
from impala.interface import Connection

from airflow.providers.common.sql.hooks.sql import DbApiHook


class ImpalaHook(DbApiHook):
"""Interact with Apache Impala through impyla."""

conn_name_attr = "impala_conn_id"
default_conn_name = "impala_default"
conn_type = "impala"
hook_name = "Impala"

def get_conn(self) -> Connection:
connection = self.get_connection(self.impala_conn_id) # pylint: disable=no-member
return connect(
host=connection.host,
port=connection.port,
user=connection.login,
password=connection.password,
database=connection.schema,
**connection.extra_dejson,
)
43 changes: 43 additions & 0 deletions airflow/providers/apache/impala/provider.yaml
@@ -0,0 +1,43 @@
# 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.

---
package-name: apache-airflow-providers-apache-impala
name: Apache Impala
description: |
`Apache Impala <https://impala.apache.org/>`__.
versions:
- 1.0.0

dependencies:
- impyla>=0.18.0,<1.0
- apache-airflow>=2.3.0

integrations:
- integration-name: Apache Impala
external-doc-url: https://impala.apache.org
tags: [apache]

hooks:
- integration-name: Apache Impala
python-modules:
- airflow.providers.apache.impala.hooks.impala

connection-types:
- hook-class-name: airflow.providers.apache.impala.hooks.impala.ImpalaHook
connection-type: impala
1 change: 1 addition & 0 deletions airflow/utils/db.py
Expand Up @@ -360,6 +360,7 @@ def create_default_connections(session: Session = NEW_SESSION):
),
session,
)
merge_conn(Connection(conn_id="impala_default", conn_type="impala", host="localhost", port=21050))
merge_conn(
Connection(
conn_id="kubernetes_default",
Expand Down
27 changes: 27 additions & 0 deletions docs/apache-airflow-providers-apache-impala/commits.rst
@@ -0,0 +1,27 @@

.. 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.
Package apache-airflow-providers-apache-impala
------------------------------------------------------

`Apache Impala <https://impala.apache.org/>`__.


This is detailed commit list of changes for versions provider package: ``apache.impala``.
For high-level changelog, see :doc:`package information including changelog <index>`.
50 changes: 50 additions & 0 deletions docs/apache-airflow-providers-apache-impala/connections/impala.rst
@@ -0,0 +1,50 @@
.. 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.
.. _howto/connection:impala:

Apache Impala Connection
========================

The Apache Impala connection type configures a connection to Apache Impala via the ``impyla`` Python package.

Default Connection IDs
----------------------

Impala hooks and operators use ``impala_default`` by default.

Configuring the Connection
--------------------------
Host (optional)
The hostname for HS2. For Impala, this can be any of the ``impalad`` service.

Port (optional)
The port number for HS2. The Impala default is ``21050``. The Hive port is
likely different.

Login (optional)
LDAP user, if applicable.

Password (optional)
LDAP password, if applicable.

Schema (optional)
The default database. If ``None``, the result is
implementation-dependent.

Extra (optional)
A JSON dictionary specifying the extra parameters that can be used in ``impyla`` connection.
48 changes: 48 additions & 0 deletions docs/apache-airflow-providers-apache-impala/index.rst
@@ -0,0 +1,48 @@
.. 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-apache-impala``
==========================================


Content
-------

.. toctree::
:maxdepth: 1
:caption: Guides

Connection Types <connections/impala>

.. toctree::
:maxdepth: 1
:caption: References

Python API<_api/airflow/providers/apache/impala/index>

.. toctree::
:maxdepth: 1
:caption: Resources

PyPI Repository <https://pypi.org/project/apache-airflow-providers-apache-impala/>
Installing from sources <installing-providers-from-sources>

.. toctree::
:maxdepth: 1
:caption: Commits

Detailed list of commits <commits>
@@ -0,0 +1,18 @@
.. 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.
.. include:: ../installing-providers-from-sources.rst
2 changes: 2 additions & 0 deletions docs/apache-airflow/extra-packages-ref.rst
Expand Up @@ -128,6 +128,8 @@ custom bash/python providers).
+---------------------+-----------------------------------------------------+------------------------------------------------+
| apache.hive | ``pip install 'apache-airflow[apache.hive]'`` | All Hive related operators |
+---------------------+-----------------------------------------------------+------------------------------------------------+
| apache.impala | ``pip install 'apache-airflow[apache.impala]'`` | All Impala related operators & hooks |
+---------------------+-----------------------------------------------------+------------------------------------------------+
| apache.kylin | ``pip install 'apache-airflow[apache.kylin]'`` | All Kylin related operators & hooks |
+---------------------+-----------------------------------------------------+------------------------------------------------+
| apache.livy | ``pip install 'apache-airflow[apache.livy]'`` | All Livy related operators, hooks & sensors |
Expand Down
9 changes: 9 additions & 0 deletions generated/provider_dependencies.json
Expand Up @@ -116,6 +116,15 @@
"vertica"
]
},
"apache.impala": {
"deps": [
"apache-airflow>=2.3.0",
"impyla>=0.18.0,<1.0"
],
"cross-providers-deps": [
"common.sql"
]
},
"apache.kylin": {
"deps": [
"apache-airflow>=2.3.0",
Expand Down
10 changes: 5 additions & 5 deletions images/breeze/output-commands-hash.txt
Expand Up @@ -2,7 +2,7 @@
# Please do not solve it but run `breeze setup regenerate-command-images`.
# This command should fix the conflict and regenerate help images that you have conflict with.
main:50fe93e07fbe182f6d374ea305841240
build-docs:903fcf1600e1348d33614d8bec816a09
build-docs:18235f12f85f8df82f3eb245e429f62d
ci:find-newer-dependencies:8fa2b57f5f0523c928743b235ee3ab5a
ci:fix-ownership:fee2c9ec9ef19686792002ae054fecdd
ci:free-space:47234aa0a60b0efd84972e6e797379f8
Expand Down Expand Up @@ -36,13 +36,13 @@ prod-image:pull:e3c89dd908fc44adf6e159c2950ebdd0
prod-image:verify:31bc5efada1d70a0a31990025db1a093
prod-image:4f98deab35e53ebddbdc9950a50555a4
release-management:generate-constraints:ae30d6ad49a1b2c15b61cb29080fd957
release-management:generate-issue-content:92b05ebd1aa7e16763f1a9a4f4714bc9
release-management:generate-issue-content:35ee0e755789aefd509f63e161aa150e
release-management:prepare-airflow-package:3ac14ea6d2b09614959c0ec4fd564789
release-management:prepare-provider-documentation:ae873854b47240fa36019a14ccc0b705
release-management:prepare-provider-packages:45d22c47b1d5e09a08d2a7218fe7a8a6
release-management:prepare-provider-documentation:452b6165f09755d052501f6003ae3ea1
release-management:prepare-provider-packages:c85c2997e01b7f9db95dc9a7abb9ea48
release-management:release-prod-images:c9bc40938e0efad49e51ef66e83f9527
release-management:verify-provider-packages:88bd609aff6d09d52ab8d80d6e055e7b
release-management:c26f14d85c5a095bebc8aa4c74c6c5fb
release-management:07fecd3ed21621993c55ac2487a412ce
setup:autocomplete:03343478bf1d0cf9c101d454cdb63b68
setup:config:3ffcd35dd24b486ddf1d08b797e3d017
setup:regenerate-command-images:ab2d83c339fa3a42b0c819b6b6cc88ae
Expand Down

0 comments on commit 6d09fc7

Please sign in to comment.