Skip to content

Commit

Permalink
Create opensearch provider and hook as base for amazon open search hook
Browse files Browse the repository at this point in the history
  • Loading branch information
cjames23 committed Sep 30, 2023
1 parent 4bafbeb commit e595762
Show file tree
Hide file tree
Showing 9 changed files with 272 additions and 69 deletions.
74 changes: 15 additions & 59 deletions airflow/providers/amazon/aws/hooks/opensearch.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,41 +23,28 @@
from opensearchpy import AWSV4SignerAuth, OpenSearch, RequestsHttpConnection

from airflow.exceptions import AirflowException
from airflow.providers.opensearch.hooks.opensearch import OpenSearchHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class OpenSearchHook(AwsBaseHook):
class AwsOpenSearchHook(OpenSearchHook, AwsBaseHook):
"""
This Hook provides a thin wrapper around the OpenSearch client.
:param: open_search_conn_id: AWS Connection to use with Open Search
:param: log_query: Whether to log the query used for Open Search
"""
conn_name_attr = "opensearch_conn_id"
default_conn_name = "opensearch_default"
conn_name_attr = "aws_opensearch_conn_id"
default_conn_name = "aws_opensearch_default"
conn_type = "opensearch"
hook_name = "AWS Open Search Hook"

def __init__(self, *args: Any, open_search_conn_id: str, log_query: bool, **kwargs: Any):
super().__init__(*args, **kwargs)
self.conn_id = open_search_conn_id
self.log_query = log_query
self.region = self.conn.extra_dejson.get("region_name", self.region_name)

conn = self.get_connection(self.conn_id)
self.use_ssl = conn.extra_dejson.get("use_ssl", False)
self.verify_certs = conn.extra_dejson.get("verify_certs", False)

self.__SERVICE = "es"
self._credentials = self.get_credentials(self.region_name)
self._auth = AWSV4SignerAuth(self._credentials, self.region_name, self.__SERVICE)

self.client = OpenSearch(
hosts=[{"host": conn.host, "port": conn.port}],
http_auth=self._auth,
use_ssl=self.use_ssl,
verify_certs=self.verify_certs,
connection_class=RequestsHttpConnection,
)
self._credentials = self.get_credentials(self.region)
self._auth = AWSV4SignerAuth(self._credentials, self.region, self.__SERVICE)

def get_client(self) -> OpenSearch:
"""
Expand All @@ -66,47 +53,15 @@ def get_client(self) -> OpenSearch:
OpenSearch client which allows using Python objects to perform searches.
"""

Check failure on line 55 in airflow/providers/amazon/aws/hooks/opensearch.py

View workflow job for this annotation

GitHub Actions / Static checks

Ruff (D205)

airflow/providers/amazon/aws/hooks/opensearch.py:50:9: D205 1 blank line required between summary line and description
self.client = OpenSearch(
hosts=[{"host": self.conn.host, "port": self.conn.port}],
http_auth=self._auth,
use_ssl=self.use_ssl,
verify_certs=self.verify_certs,
connection_class=RequestsHttpConnection,
)
return self.client

def search(self, query: dict, index_name: str, **kwargs: Any) -> Any:
"""
Runs a search query against the connected OpenSearch cluster.
:param: query: The query for the search against OpenSearch.
:param: index_name: The name of the index to search against
"""
if self.log_query:
self.log.info("Searching %s with Query: %s", index_name, query)
return self.client.search(body=query, index=index_name, **kwargs)

def index(self, document: dict, index_name: str, doc_id: int, **kwargs: Any) -> Any:
"""
Index a document on open search.
:param: document: A dictionary representation of the document
:param: index_name: the name of the index that this document will be associated with
:param: doc_id: the numerical identifier that will be used to identify the document on the index.
"""
return self.client.index(index=index_name, id=doc_id, body=document, **kwargs)

def delete(self, index_name: str, query: dict | None = None, doc_id: int | None = None):
"""
Delete from an index by either a query or by the document id.
:param: index_name: the name of the index to delete from
:param: query: If deleting by query a dict representation of the query to run to
identify documents to delete.
:param: doc_id: The identifier of the document to delete.
"""
if query is not None:
if self.log_query:
self.log.info("Deleting from %s using Query: %s", index_name, query)
return self.client.delete_by_query(index=index_name, body=query)
elif doc_id is not None:
return self.client.delete(index=index_name, id=doc_id)
else:
AirflowException("To delete a document you must include one of either a query or a document id. ")

@staticmethod
def get_ui_field_behaviour() -> dict[str, Any]:
"""Returns custom UI field behaviour for Amazon Open Search Connection."""
Expand All @@ -123,6 +78,7 @@ def get_ui_field_behaviour() -> dict[str, Any]:
{
"use_ssl": True,
"verify_certs": True,
"region_name": "us-east-1"
},
indent=2,
),
Expand Down
3 changes: 1 addition & 2 deletions airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ dependencies:
- apache-airflow>=2.4.0
- apache-airflow-providers-common-sql>=1.3.1
- apache-airflow-providers-http
- apache-airflow-providers-opensearch
# We should update minimum version of boto3 and here regularly to avoid `pip` backtracking with the number
# of candidates to consider. We should also make sure that all the below related packages have also the
# same minimum version specified. Boto3 1.28.0 has been released on July 6, 2023. We should also make
Expand All @@ -94,8 +95,6 @@ dependencies:
- mypy-boto3-appflow>=1.24.0
- asgiref
- mypy-boto3-s3>=1.24.0
- opensearch-py>=2.2.0
- opensearch-dsl>=2.1.0


integrations:
Expand Down
27 changes: 27 additions & 0 deletions airflow/providers/opensearch/CHANGELOG.rst
Original file line number Diff line number Diff line change
@@ -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.
.. NOTE TO CONTRIBUTORS:
Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes
and you want to add an explanation to the users on how they are supposed to deal with them.
The changelog is updated and maintained semi-automatically by release manager.
``apache-airflow-providers-opensearch``

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

import json
from functools import cached_property
from typing import Any

from opensearchpy import OpenSearch, RequestsHttpConnection

from airflow.exceptions import AirflowException
from airflow.hooks.base import BaseHook


class OpenSearchHook(BaseHook):
"""
This Hook provides a thin wrapper around the OpenSearch client.
:param: open_search_conn_id: Connection to use with Open Search
:param: log_query: Whether to log the query used for Open Search
"""
conn_name_attr = "opensearch_conn_id"
default_conn_name = "opensearch_default"
conn_type = "opensearch"
hook_name = "OpenSearch Hook"

def __init__(self, *args: Any, open_search_conn_id: str, log_query: bool, **kwargs: Any):
super().__init__(*args, **kwargs)
self.client = None
self.conn_id = open_search_conn_id
self.log_query = log_query

self.conn = self.get_connection(self.conn_id)
self.use_ssl = self.conn.extra_dejson.get("use_ssl", False)
self.verify_certs = self.conn.extra_dejson.get("verify_certs", False)
self.__SERVICE = "es"

@cached_property
def get_client(self) -> OpenSearch:
"""
This function is intended for Operators that will take in arguments and use the high level
OpenSearch client which allows using Python objects to perform searches.
"""
auth = (self.conn.login, self.conn.password)

Check failure on line 61 in airflow/providers/opensearch/hooks/opensearch.py

View workflow job for this annotation

GitHub Actions / Static checks

Ruff (D205)

airflow/providers/opensearch/hooks/opensearch.py:56:9: D205 1 blank line required between summary line and description
self.client = OpenSearch(
hosts=[{"host": self.conn.host, "port": self.conn.port}],
http_auth=auth,
use_ssl=self.use_ssl,
verify_certs=self.verify_certs,
connection_class=RequestsHttpConnection,
)
return self.client

def search(self, query: dict, index_name: str, **kwargs: Any) -> Any:
"""
Runs a search query against the connected OpenSearch cluster.
:param: query: The query for the search against OpenSearch.
:param: index_name: The name of the index to search against
"""
if self.log_query:
self.log.info("Searching %s with Query: %s", index_name, query)
return self.client.search(body=query, index=index_name, **kwargs)

def index(self, document: dict, index_name: str, doc_id: int, **kwargs: Any) -> Any:
"""
Index a document on open search.
:param: document: A dictionary representation of the document
:param: index_name: the name of the index that this document will be associated with
:param: doc_id: the numerical identifier that will be used to identify the document on the index.
"""
return self.client.index(index=index_name, id=doc_id, body=document, **kwargs)

def delete(self, index_name: str, query: dict | None = None, doc_id: int | None = None):
"""
Delete from an index by either a query or by the document id.
:param: index_name: the name of the index to delete from
:param: query: If deleting by query a dict representation of the query to run to
identify documents to delete.
:param: doc_id: The identifier of the document to delete.
"""
if query is not None:
if self.log_query:
self.log.info("Deleting from %s using Query: %s", index_name, query)
return self.client.delete_by_query(index=index_name, body=query)
elif doc_id is not None:
return self.client.delete(index=index_name, id=doc_id)
else:
AirflowException("To delete a document you must include one of either a query or a document id. ")

@staticmethod
def get_ui_field_behaviour() -> dict[str, Any]:
"""Returns custom UI field behaviour for Open Search Connection."""
return {
"hidden_fields": ["schema"],
"relabeling": {
"extra": "Open Search Configuration",
},
"placeholders": {
"extra": json.dumps(
{
"use_ssl": True,
"verify_certs": True
},
indent=2,
),
},
}
46 changes: 46 additions & 0 deletions airflow/providers/opensearch/provider.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
# 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-opensearch
name: Opensearch
description: |
`Opensearch <https://opensearch.org/>`__
suspended: false
versions:
- 1.0.0

dependencies:
- apache-airflow>=2.4.3
- opensearch-py>=2.2.0
- opensearch-dsl>=2.1.0

integrations:
- integration-name: Opensearch
external-doc-url: https://opensearch.org/
logo: /integration-logos/opensearch/opensearch.png
tags: [software]

hooks:
- integration-name: Opensearch
python-modules:
- airflow.providers.opensearch.hooks.opensearch

connection-types:
- hook-class-name: airflow.providers.opensearch.hooks.opensearch.OpenSearchHook
connection-type: opensearch
29 changes: 24 additions & 5 deletions tests/providers/amazon/aws/hooks/test_opensearch.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,33 +17,52 @@
from __future__ import annotations

import pytest
from moto import mock_opensearch
import boto3

from airflow.models import Connection
from airflow.providers.amazon.aws.hooks.opensearch import OpenSearchHook
from airflow.utils import db


class TestOpenSearchHook:
@mock_opensearch
def create_domain(self):
client = boto3.client("opensearch")
response = client.create_domain(DomainName=f"test-opensearch-cluster",
EngineVersion="2.7",
ClusterConfig={
"InstanceType": "t3.small.search",
"InstanceCount": 1,
"DedicatedMasterEnabled": False,
"ZoneAwarenessEnabled": False,
},)
return response["endpoint"]

def setup_method(self):
db.merge_conn(
Connection(
conn_id="open_search_conn",
conn_type="open_search",
host="myhost.opensearch.com",
conn_id="opensearch_default",
conn_type="opensearch",
host=self.create_domain(),
login="MyAWSSecretID",
password="MyAccessKey",
extra={
"region_name": "us-east-1"
}
)
)

@pytest.fixture()
def mock_search(self, monkeypatch):
def mock_return():
def mock_return(index_name: str):
return {"status": "test"}

monkeypatch.setattr(OpenSearchHook, "search", mock_return)

def test_hook_search(self, mock_search):
hook = OpenSearchHook(open_search_conn_id="open_search_conn", log_query=True)
hook = OpenSearchHook(open_search_conn_id="opensearch_default",
log_query=True)

result = hook.search(
index_name="testIndex",
Expand Down
Loading

0 comments on commit e595762

Please sign in to comment.