diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml index 8a03154197c30cadb3a8400608b6fe9babe70776..0f0d2997e78b2e45e170719b61b7a29d892aeaa4 100644 --- a/.gitlab-ci.yml +++ b/.gitlab-ci.yml @@ -66,7 +66,7 @@ unit_tests: image: eu.gcr.io/osdu-cicd-epam/airflow-python-dags/airflow-python-dags:latest script: - chmod +x tests/unit_tests.sh - - tests/./unit_tests.sh || EXIT_CODE=$? + # - tests/./unit_tests.sh || EXIT_CODE=$? #TODO: unit tests moved to airflow-osdu-lib, need to remove `unit_tests` CI step later - exit ${EXIT_CODE} # TODO: Think about how rsync must look. At the moment it looks messy. diff --git a/README.md b/README.md index d0f81b16b44481068fd2354ebca7dffd8d258ce0..99fb773570fd6192b1bd3b55900d4ca1e8f718a2 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,6 @@ * * [Ingestion](#ingestion) * * [OSDU Python SDK](#osdu-python-sdk) * [Testing](#testing) -* * [Running Unit Tests](#running-unit-tests) * * [Running E2E Tests](#running-e2e-tests) * [Logging](#logging) * * [Logging Configuration](#logging-configuration) @@ -129,15 +128,6 @@ If variable defines URL to internal services it should have suffix which show th | core__ingestion__batch_save_size | Size of the batch of entities to save in Storage Service| -## Testing -### Running Unit Tests -~~~ -tests/./set_airflow_env.sh -~~~ -~~~ -chmod +x tests/test_dags.sh && tests/./test_dags.sh -~~~ - ### Running E2E Tests ~~~ tests/./set_airflow_env.sh diff --git a/src/osdu_dags/osdu-ingest-r2.py b/src/osdu_dags/osdu-ingest-r2.py index 1d81989d4c6be5b7bd9062bb06753f0faf727fdf..a9a406b437a36f3e1f546fb9b36834fd907c642e 100644 --- a/src/osdu_dags/osdu-ingest-r2.py +++ b/src/osdu_dags/osdu-ingest-r2.py @@ -20,9 +20,9 @@ from datetime import timedelta import airflow from airflow import DAG from osdu_airflow.backward_compatibility.default_args import update_default_args -from osdu_manifest.operators.deprecated.update_status import UpdateStatusOperator -from osdu_manifest.operators.process_manifest_r2 import ProcessManifestOperatorR2 -from osdu_manifest.operators.search_record_id import SearchRecordIdOperator +from osdu_airflow.operators.deprecated.update_status import UpdateStatusOperator +from osdu_airflow.operators.process_manifest_r2 import ProcessManifestOperatorR2 +from osdu_airflow.operators.search_record_id import SearchRecordIdOperator default_args = { "start_date": airflow.utils.dates.days_ago(0), diff --git a/src/osdu_dags/osdu-ingest-r3.py b/src/osdu_dags/osdu-ingest-r3.py index e84b32843d2db1d4629efc0839354f95b0588438..3819fbc3fa5af4a9458cb790b1b76219065ecb05 100644 --- a/src/osdu_dags/osdu-ingest-r3.py +++ b/src/osdu_dags/osdu-ingest-r3.py @@ -23,11 +23,11 @@ from airflow.models import Variable from airflow.operators.dummy_operator import DummyOperator from airflow.operators.python_operator import BranchPythonOperator from osdu_airflow.backward_compatibility.default_args import update_default_args +from osdu_airflow.operators.ensure_manifest_integrity import EnsureManifestIntegrityOperator +from osdu_airflow.operators.process_manifest_r3 import ProcessManifestOperatorR3 +from osdu_airflow.operators.update_status import UpdateStatusOperator +from osdu_airflow.operators.validate_manifest_schema import ValidateManifestSchemaOperator from osdu_api.libs.exceptions import NotOSDUSchemaFormatError -from osdu_manifest.operators.ensure_manifest_integrity import EnsureManifestIntegrityOperator -from osdu_manifest.operators.process_manifest_r3 import ProcessManifestOperatorR3 -from osdu_manifest.operators.update_status import UpdateStatusOperator -from osdu_manifest.operators.validate_manifest_schema import ValidateManifestSchemaOperator BATCH_NUMBER = int(Variable.get("core__ingestion__batch_count", "3")) PROCESS_SINGLE_MANIFEST_FILE = "process_single_manifest_file_task" diff --git a/src/osdu_dags/osdu_manifest/hooks/__init__.py b/src/osdu_dags/osdu_manifest/hooks/__init__.py deleted file mode 100644 index 5c873a4f1c5a04c0f395e3d1e9106c3cfba0cf7a..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/hooks/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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 .http_hooks import * diff --git a/src/osdu_dags/osdu_manifest/hooks/http_hooks.py b/src/osdu_dags/osdu_manifest/hooks/http_hooks.py deleted file mode 100644 index cc747c167aca5c799008038ed4c9068f63840738..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/hooks/http_hooks.py +++ /dev/null @@ -1,21 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - -"""Http Hooks.""" - -from airflow.hooks.http_hook import HttpHook - -workflow_hook = HttpHook(http_conn_id='workflow', method="POST") -search_http_hook = HttpHook(http_conn_id='search', method="POST") diff --git a/src/osdu_dags/osdu_manifest/operators/__init__.py b/src/osdu_dags/osdu_manifest/operators/__init__.py deleted file mode 100644 index 5511adb864013540d7eda3007c4ffb6186bc68cb..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/__init__.py +++ /dev/null @@ -1,14 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. diff --git a/src/osdu_dags/osdu_manifest/operators/deprecated/__init__.py b/src/osdu_dags/osdu_manifest/operators/deprecated/__init__.py deleted file mode 100644 index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000 diff --git a/src/osdu_dags/osdu_manifest/operators/deprecated/update_status.py b/src/osdu_dags/osdu_manifest/operators/deprecated/update_status.py deleted file mode 100644 index d19d68314a1422f511d92c58e755fc3ec44e9548..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/deprecated/update_status.py +++ /dev/null @@ -1,95 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - -"""Update Status operator.""" - -import copy -import enum -import logging - -from airflow.models import BaseOperator, Variable -from osdu_api.libs.context import Context -from osdu_api.libs.exceptions import PipelineFailedError -from osdu_api.libs.refresh_token import AirflowTokenRefresher -from osdu_api.libs.update_status import UpdateStatus - -logger = logging.getLogger() - - -class UpdateStatusOperator(BaseOperator): - """Operator to update status.""" - ui_color = '#10ECAA' - ui_fgcolor = '#000000' - - class prev_ti_state(enum.Enum): - NONE = "running" - SUCCESS = "finished" - FAILED = "failed" - - def get_previous_ti_statuses(self, context: dict) -> enum.Enum: - """Get status of previous tasks' executions. - Return corresponding enum value. - - :param context: Airflow context - :type context: dict - :return: Previous status - :rtype: enum.Enum - """ - dagrun = context['ti'].get_dagrun() - failed_ti = dagrun.get_task_instances(state='failed') - success_ti = dagrun.get_task_instances(state='success') - if not failed_ti and not success_ti: # There is no prev task so it can't have been failed - logger.info("There are no tasks before this one. So it has status RUNNING") - return self.prev_ti_state.NONE - if failed_ti: - logger.info("There are failed tasks before this one. So it has status FAILED") - return self.prev_ti_state.FAILED - logger.info("There are successed tasks before this one. So it has status SUCCESSED") - return self.prev_ti_state.SUCCESS - - def pre_execute(self, context: dict): - self.status = self.get_previous_ti_statuses(context) - - def execute(self, context: dict): - """Execute update workflow status. - If status assumed to be FINISHED then we check whether records - are searchable or not. - If they are then update status FINISHED else FAILED - - :param context: Airflow context - :type context: dict - :raises PipelineFailedError: If any of the status is failed - """ - conf = copy.deepcopy(context["dag_run"].conf) - logger.debug(f"Got conf {conf}.") - if "Payload" in conf: - payload_context = Context.populate(conf) - else: - payload_context = Context(data_partition_id=conf["data-partition-id"], - app_key=conf.get("AppKey", "")) - workflow_id = conf["WorkflowID"] - status = self.status.value - status_updater = UpdateStatus( - workflow_name="", - workflow_url=Variable.get("core__service__workflow__url"), - workflow_id=workflow_id, - run_id="", - status=status, - token_refresher=AirflowTokenRefresher(), - context=payload_context - ) - status_updater.update_workflow_status() - if self.status is self.prev_ti_state.FAILED: - raise PipelineFailedError("Dag failed") diff --git a/src/osdu_dags/osdu_manifest/operators/ensure_manifest_integrity.py b/src/osdu_dags/osdu_manifest/operators/ensure_manifest_integrity.py deleted file mode 100644 index 99ed2667e3d6794fe4c4d2f23a22a5c4419d2b04..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/ensure_manifest_integrity.py +++ /dev/null @@ -1,79 +0,0 @@ -# Copyright 2021 Google LLC -# Copyright 2021 EPAM Systems -# -# Licensed 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. - -"""R3 Validate reference Manifest operator.""" -import logging - -from airflow.models import BaseOperator, Variable -from osdu_airflow.backward_compatibility.airflow_utils import apply_defaults -from osdu_api.libs.context import Context -from osdu_api.libs.refresh_token import AirflowTokenRefresher -from osdu_api.libs.validation.validate_file_source import FileSourceValidator -from osdu_api.libs.validation.validate_referential_integrity import ManifestIntegrity -from osdu_manifest.operators.mixins.ReceivingContextMixin import ReceivingContextMixin - -logger = logging.getLogger() - - -class EnsureManifestIntegrityOperator(BaseOperator, ReceivingContextMixin): - """Operator to validate ref inside manifest R3 and remove invalid entities.""" - - ui_color = '#dad5ff' - ui_fgcolor = '#000000' - - @apply_defaults - def __init__(self, - previous_task_id: str=None, - *args, **kwargs): - """Init base operator and obtain base urls from Airflow Variables.""" - super().__init__(*args, **kwargs) - self.search_url = Variable.get('core__service__search__url') - self.whitelist_ref_patterns = Variable.get('core__config__reference_patterns_whitelist', default_var=None) - self.previous_task_id = previous_task_id - self._show_skipped_ids = Variable.get( - 'core__config__show_skipped_ids', default_var=False - ) - - def execute(self, context: dict): - """Execute manifest validation then process it. - - :param context: Airflow context - :type context: dict - """ - payload_context = Context.populate(context["dag_run"].conf["execution_context"]) - token_refresher = AirflowTokenRefresher() - file_source_validator = FileSourceValidator() - - manifest_integrity = ManifestIntegrity( - self.search_url, - token_refresher, - file_source_validator, - payload_context, - self.whitelist_ref_patterns, - ) - - execution_context = context["dag_run"].conf["execution_context"] - manifest_data = self._get_manifest_data(context, execution_context) - previously_skipped_entities = self._get_previously_skipped_entities(context) - logger.debug(f"Manifest data: {manifest_data}") - - manifest, skipped_ids = manifest_integrity.ensure_integrity( - manifest_data, - previously_skipped_entities - ) - logger.debug(f"Valid manifest data: {manifest_data}") - if self._show_skipped_ids: - context["ti"].xcom_push(key="skipped_ids", value=skipped_ids) - return {"manifest": manifest} diff --git a/src/osdu_dags/osdu_manifest/operators/mixins/ReceivingContextMixin.py b/src/osdu_dags/osdu_manifest/operators/mixins/ReceivingContextMixin.py deleted file mode 100644 index 614f1006b29911b73835255b561554e7777bd611..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/mixins/ReceivingContextMixin.py +++ /dev/null @@ -1,51 +0,0 @@ -# Copyright 2021 Google LLC -# Copyright 2021 EPAM Systems -# -# Licensed 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. - -import logging - -from osdu_api.libs.types import ManifestType - - -class ReceivingContextMixin: - """Mixin for receiving manifest file from XCOMs in case if current operator not the first in the row""" - - def _get_manifest_data(self, context: dict, execution_context: dict) -> ManifestType: - """ - Receive manifest file. If previous task id not None - get manifest file from XCOMs. - Otherwise - get manifest file from execution context - """ - if self.previous_task_id: - previous_task_value = context["ti"].xcom_pull(task_ids=self.previous_task_id, - key="return_value") - if previous_task_value: - manifest_data = previous_task_value["manifest"] - else: - manifest_data = execution_context["manifest"] - else: - manifest_data = execution_context["manifest"] - return manifest_data - - def _get_previously_skipped_entities(self, context: dict) -> list: - """ - Receive skipped entities from previous tasks. - """ - previously_skipped_ids = [] - dagrun = context['ti'].get_dagrun() - task_instances = dagrun.get_task_instances() - for task in task_instances: - task_skipped_ids = context["ti"].xcom_pull(key="skipped_ids", task_ids=task.task_id) - if task_skipped_ids: - previously_skipped_ids.extend(task_skipped_ids) - return previously_skipped_ids diff --git a/src/osdu_dags/osdu_manifest/operators/mixins/__init__.py b/src/osdu_dags/osdu_manifest/operators/mixins/__init__.py deleted file mode 100644 index de8f5ce5bf56bdd78824065dbbd52846984960b1..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/mixins/__init__.py +++ /dev/null @@ -1,14 +0,0 @@ -# Copyright 2021 Google LLC -# Copyright 2021 EPAM Systems -# -# Licensed 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. diff --git a/src/osdu_dags/osdu_manifest/operators/process_manifest_r2.py b/src/osdu_dags/osdu_manifest/operators/process_manifest_r2.py deleted file mode 100644 index 79ee1d3c5dffdca1442972a775af85e9a4f2b6f3..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/process_manifest_r2.py +++ /dev/null @@ -1,381 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - -"""R2 Process Manifest operator.""" - -import configparser -import enum -import json -import logging -import re -import sys -import time -import uuid -from collections import Counter -from typing import Tuple -from urllib.error import HTTPError - -import requests -import tenacity -from airflow.models import BaseOperator, Variable -from osdu_api.libs.auth.authorization import authorize -from osdu_api.libs.refresh_token import AirflowTokenRefresher - -config = configparser.RawConfigParser() -config.read(Variable.get("core__config__dataload_config_path")) - -DEFAULT_TENANT = config.get("DEFAULTS", "tenant") -DEFAULT_SOURCE = config.get("DEFAULTS", "authority") -DEFAULT_VERSION = config.get("DEFAULTS", "kind_version") - -RETRIES = 3 -TIMEOUT = 1 - -# Set up base logger -handler = logging.StreamHandler(sys.stdout) -handler.setFormatter( - logging.Formatter("%(asctime)s [%(name)-14.14s] [%(levelname)-7.7s] %(message)s")) -logger = logging.getLogger("Dataload") -logger.setLevel(logging.INFO) -logger.addHandler(handler) - -# Some constants, used by script -SEARCH_OK_RESPONSE_CODES = [200] -DATA_LOAD_OK_RESPONSE_CODES = [201] -NOT_FOUND_RESPONSE_CODES = [404] -BAD_TOKEN_RESPONSE_CODES = [400, 401, 403, 500] - - -class FileType(enum.Enum): - MANIFEST = enum.auto() - WORKPRODUCT = enum.auto() - - -def dataload(**kwargs): - data_conf = kwargs['dag_run'].conf - conf_payload = kwargs["dag_run"].conf["Payload"] - loaded_conf = { - "acl": conf_payload["acl"], - "legal_tag": conf_payload["legal"], - "data_object": data_conf - } - return loaded_conf, conf_payload - - -def create_headers(conf_payload): - """Create header. - - :param conf_payload: config payload - :return: headers - """ - partition_id = conf_payload["data-partition-id"] - app_key = conf_payload["AppKey"] - headers = { - 'Content-type': 'application/json', - 'data-partition-id': partition_id, - 'AppKey': app_key - } - return headers - - -def generate_id(type_id): - """Generate resource ID. - - :param type_id: resource type ID - :return: resource ID - """ - return "{0}{1}:".format(type_id.replace("type:", ""), re.sub(r"\D", "", str(uuid.uuid4()))) - - -def determine_data_type(raw_resource_type_id): - """Determine resource type ID. - - :param raw_resource_type_id: raw resource type ID from manifest file - :return: short resource type ID - """ - return raw_resource_type_id.split("/")[-1].replace(":", "") \ - if raw_resource_type_id is not None else None - - -# TODO: add comments to functions that implement actions in this function -def process_file_items(loaded_conf, conf_payload) -> Tuple[list, list]: - """Process files items. - - :param loaded_conf: loaded configuration - :param conf_payload: configuration payload - :return: list of file records and list of their ids - """ - file_ids = [] - file_list = [] - data_object = loaded_conf.get("data_object") - acl = loaded_conf.get("acl") - legal_tag = loaded_conf.get("legal_tag") - for file in data_object["Files"]: - file["ResourceID"] = generate_id(file["ResourceTypeID"]) - file_ids.append(file["ResourceID"]) - file_list.append( - ( - populate_request_body(file, acl, legal_tag, "file", conf_payload), - "File" - ) - ) - return file_list, file_ids - - -def process_wpc_items(loaded_conf, product_type, file_ids, conf_payload): - """Process WorkProductComponents items. - - :param loaded_conf: loaded configuration - :param product_type: product type - :param file_ids: list of file ids - :param conf_payload: configuration payload - :return: list of workproductcomponents records and list of their ids - """ - wpc_ids = [] - wpc_list = [] - data_object = loaded_conf.get("data_object") - acl = loaded_conf.get("acl") - legal_tag = loaded_conf.get("legal_tag") - for wpc in data_object["WorkProductComponents"]: - wpc["ResourceID"] = generate_id(wpc["ResourceTypeID"]) - wpc_ids.append(wpc["ResourceID"]) - wpc["Data"]["GroupTypeProperties"]["Files"] = file_ids - wpc_list.append( - ( - populate_request_body(wpc, acl, legal_tag, product_type + "_wpc", conf_payload), - product_type + "_wpc" - ) - ) - return wpc_list, wpc_ids - - -def process_wp_item(loaded_conf, product_type, wpc_ids, conf_payload) -> list: - """Process WorkProduct item. - - :param loaded_conf: loaded configuration - :param product_type: product type - :param wpc_ids: work product component ids - :param conf_payload: configuration payload - :return: work product record - """ - data_object = loaded_conf.get("data_object") - acl = loaded_conf.get("acl") - legal_tag = loaded_conf.get("legal_tag") - - work_product = data_object["WorkProduct"] - work_product["ResourceID"] = generate_id(work_product["ResourceTypeID"]) - work_product["Data"]["GroupTypeProperties"]["Components"] = wpc_ids - work_product = [ - ( - populate_request_body(work_product, acl, legal_tag, product_type + "_wp", conf_payload), - product_type + "_wp" - ) - ] - return work_product - - -def validate_file_type(file_type, data_object): - """Validate file type. - - :param file_type: file type - :param data_object: file record - """ - if not file_type: - logger.error(f"Error with file {data_object}. Type could not be specified.") - sys.exit(2) - - -def validate_file(loaded_conf) -> Tuple[FileType, str]: - """Validate file. - - :param loaded_conf: loaded configuration - :return: file type and produc type - """ - data_object = loaded_conf.get("data_object") - if not data_object: - logger.error(f"Error with file {data_object}. It is empty.") - sys.exit(2) - elif "Manifest" in data_object and "ResourceTypeID" in data_object.get("Manifest"): - product_type = determine_data_type(data_object["Manifest"].get("ResourceTypeID")) - validate_file_type(product_type, data_object) - return (FileType.MANIFEST, product_type) - elif "WorkProduct" in data_object and "ResourceTypeID" in data_object.get("WorkProduct"): - product_type = determine_data_type(data_object["WorkProduct"].get("ResourceTypeID")) - validate_file_type(product_type, data_object) - if product_type.lower() == "workproduct" and \ - data_object.get("WorkProductComponents") and \ - len(data_object["WorkProductComponents"]) >= 1: - product_type = determine_data_type( - data_object["WorkProductComponents"][0].get("ResourceTypeID")) - validate_file_type(product_type, data_object) - return (FileType.WORKPRODUCT, product_type) - else: - logger.error( - f"Error with file {data_object}. It doesn't have either Manifest or WorkProduct or ResourceTypeID.") - sys.exit(2) - - -def create_kind(data_kind, conf_payload): - """Create kind. - - :param data_kind: data kind - :param conf_payload: configuration payload - :return: kind - """ - partition_id = conf_payload.get("data-partition-id", DEFAULT_TENANT) - source = conf_payload.get("authority", DEFAULT_SOURCE) - version = conf_payload.get("kind_version", DEFAULT_VERSION) - kind_init = config.get("KINDS_INITIAL", f"{data_kind.lower()}_kind") - kind = f"{partition_id}:{source}:{kind_init}:{version}" - return kind - - -def populate_request_body(data, acl, legal_tag, data_type, conf_payload): - """Populate request body according API specification - - :param data: item data from manifest files - :param data_type: resource type ID - :return: populated request - :rtype: dict - """ - request = {"kind": create_kind(data_type, conf_payload), - "legal": { - "legaltags": [], - "otherRelevantDataCountries": ["US"], - "status": "compliant" - }, - "acl": { - "viewers": [], - "owners": [] - }, - "data": data} - request["legal"]["legaltags"] = legal_tag["legaltags"] - request["acl"]["viewers"] = acl["viewers"] - request["acl"]["owners"] = acl["owners"] - return request - - -def separate_type_data(request_data): - """Separate the list of tuples into Data Type Counter and data list - - :param request_data: tuple of data and types - :type request_data: tuple(list, str) - :return: counter with data types and data list - :rtype: tuple(counter, list) - """ - data = [] - types = Counter() - for elem in request_data: - data.append(elem[0]) - types[elem[1]] += 1 - logger.info(f"The count of records to be ingested: {str(dict(types))}") - return types, data - - -def create_manifest_request_data(loaded_conf: dict, product_type: str): - acl = loaded_conf.get("acl") - legal_tag = loaded_conf.get("legal_tag") - data_object = loaded_conf.get("data_object") - data_objects_list = [ - ( - populate_request_body(data_object["Manifest"], acl, legal_tag, product_type), - product_type)] - return data_objects_list - - -def create_workproduct_request_data(loaded_conf: dict, product_type: str, wp, wpc_list, file_list): - data_object_list = file_list + wpc_list + wp - types, data_objects_list = separate_type_data(data_object_list) - return data_objects_list - - -@tenacity.retry( - wait=tenacity.wait_fixed(TIMEOUT), - stop=tenacity.stop_after_attempt(RETRIES), - reraise=True -) -@authorize(AirflowTokenRefresher()) -def send_request(headers, request_data): - """Send request to records storage API.""" - - logger.error(f"Header {str(headers)}") - # loop for implementing retries send process - retries = RETRIES - - for retry in range(retries): - try: - # send batch request for creating records - response = requests.put(Variable.get('core__service__storage__url'), json.dumps(request_data), - headers=headers) - - if response.status_code in DATA_LOAD_OK_RESPONSE_CODES: - logger.info(",".join(map(str, response.json()["recordIds"]))) - return response - - reason = response.text[:250] - logger.error(f"Request error.") - logger.error(f"Response status: {response.status_code}. " - f"Response content: {reason}.") - - if retry + 1 < retries: - if response.status_code in BAD_TOKEN_RESPONSE_CODES: - logger.error("Invalid or expired token.") - return response - else: - time_to_sleep = TIMEOUT - - logger.info(f"Retrying in {time_to_sleep} seconds...") - time.sleep(time_to_sleep) - - except (requests.RequestException, HTTPError) as exc: - logger.error(f"Unexpected request error. Reason: {exc}") - sys.exit(2) - - - # End script if ran out of retries and data could not be uploaded. - else: - logger.error(f"Request could not be completed.\n" - f"Reason: {reason}") - sys.exit(2) - - -def process_manifest(**kwargs): - """Process manifest.""" - loaded_conf, conf_payload = dataload(**kwargs) - file_type, product_type = validate_file(loaded_conf) - if file_type is FileType.MANIFEST: - manifest_record = create_manifest_request_data(loaded_conf, product_type) - elif file_type is FileType.WORKPRODUCT: - file_list, file_ids = process_file_items(loaded_conf, conf_payload) - kwargs["ti"].xcom_push(key="file_ids", value=file_ids) - wpc_list, wpc_ids = process_wpc_items(loaded_conf, product_type, file_ids, conf_payload) - wp_list = process_wp_item(loaded_conf, product_type, wpc_ids, conf_payload) - manifest_record = create_workproduct_request_data(loaded_conf, product_type, wp_list, - wpc_list, - file_list) - else: - sys.exit(2) - headers = create_headers(conf_payload) - record_ids = send_request(headers, request_data=manifest_record).json()["recordIds"] - kwargs["ti"].xcom_push(key="record_ids", value=record_ids) - - -class ProcessManifestOperatorR2(BaseOperator): - """R2 Manifest Operator.""" - ui_color = '#dad5ff' - ui_fgcolor = '#000000' - - def execute(self, context): - process_manifest(**context) diff --git a/src/osdu_dags/osdu_manifest/operators/process_manifest_r3.py b/src/osdu_dags/osdu_manifest/operators/process_manifest_r3.py deleted file mode 100644 index e05412340c42e0669359cabdec87028d565f1c43..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/process_manifest_r3.py +++ /dev/null @@ -1,176 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - - -""" -R3 Process Manifest operator. -""" - -import logging -from math import ceil -from typing import List, Tuple - -from airflow.models import BaseOperator, Variable -from jsonschema import SchemaError -from osdu_api.libs.constants import DATA_TYPES_WITH_SURROGATE_KEYS, SURROGATE_KEYS_PATHS -from osdu_api.libs.context import Context -from osdu_api.libs.exceptions import (EmptyManifestError, GenericManifestSchemaError, - GetSchemaError, NotOSDUSchemaFormatError, UploadFileError) -from osdu_api.libs.handle_file import FileHandler -from osdu_api.libs.process_manifest_r3 import ManifestProcessor -from osdu_api.libs.processors.single_manifest_processor import SingleManifestProcessor -from osdu_api.libs.refresh_token import AirflowTokenRefresher -from osdu_api.libs.search_client import SearchClient -from osdu_api.libs.source_file_check import SourceFileChecker -from osdu_api.libs.types import ManifestType -from osdu_api.libs.validation.validate_file_source import FileSourceValidator -from osdu_api.libs.validation.validate_referential_integrity import ManifestIntegrity -from osdu_api.libs.validation.validate_schema import SchemaValidator -from osdu_airflow.backward_compatibility.airflow_utils import apply_defaults -from osdu_manifest.operators.mixins.ReceivingContextMixin import ReceivingContextMixin -from requests import HTTPError - -logger = logging.getLogger() - - -class ProcessManifestOperatorR3(BaseOperator, ReceivingContextMixin): - """Operator to process manifest R3.""" - - ui_color = '#dad5ff' - ui_fgcolor = '#000000' - - @apply_defaults - def __init__(self, previous_task_id: str = None, batch_number=3, *args, **kwargs): - """Init base operator and obtain base urls from Airflow Variables.""" - super().__init__(*args, **kwargs) - self.previous_task_id = previous_task_id - self.batch_number = batch_number - self.schema_service_url = Variable.get('core__service__schema__url') - self.search_service_url = Variable.get('core__service__search__url') - self.storage_url = Variable.get('core__service__storage__url') - self.file_service_host = Variable.get('core__service__file__host') - self.batch_count = int(Variable.get("core__ingestion__batch_count", "3")) - self._show_skipped_ids = Variable.get('core__config__show_skipped_ids', default_var=False) - - def _get_manifest_files_range(self, manifests: List[dict]) -> Tuple[int, int]: - """ - Get start and end indexes of a manifest files slice to be processed within this task. - :param manifests: A list of manifests - :return: start index, end index - """ - split_size = ceil(len(manifests) / self.batch_count) - slice_start_index = (self.batch_number - 1) * split_size - slice_end_index = self.batch_number * split_size - return slice_start_index, slice_end_index - - def _process_manifest( - self, - single_manifest_processor: SingleManifestProcessor, - manifest: ManifestType - ) -> Tuple[List[str], List[dict]]: - """ - - :param single_manifest_processor: Object to process a single manifest file. - Processing includes validation against schemas, storing records enc. - :param manifest: A single manifest file or a list of them. - :return: - """ - skipped_entities = [] - if isinstance(manifest, dict): - record_ids, skipped_entities = single_manifest_processor.process_manifest( - manifest, False) - - elif isinstance(manifest, list): - record_ids = [] - slice_start_index, slice_end_index = self._get_manifest_files_range(manifest) - logger.debug(f"Start and indexes {slice_start_index}:{slice_end_index}") - for single_manifest in manifest[slice_start_index:slice_end_index]: - logger.debug(f"processing {single_manifest}") - try: - saved_records_ids, not_saved_records = single_manifest_processor.process_manifest( - single_manifest, True - ) - record_ids.extend(saved_records_ids) - skipped_entities.extend(not_saved_records) - except (UploadFileError, HTTPError, GetSchemaError, SchemaError, - GenericManifestSchemaError) as e: - logger.error(f"Can't process {single_manifest}") - logger.error(e) - continue - else: - raise NotOSDUSchemaFormatError( - f"Manifest {manifest} must be either not empty 'list' or 'dict'") - - return record_ids, skipped_entities - - def execute(self, context: dict): - """Execute manifest validation then process it. - - Get a single manifest file or a list of them. - If it is a list, calculate which range (slice) of manifest files must be processed and then - process this range one by one. - - :param context: Airflow context - :type context: dict - """ - execution_context = context["dag_run"].conf["execution_context"] - payload_context = Context.populate(execution_context) - token_refresher = AirflowTokenRefresher() - file_handler = FileHandler(self.file_service_host, token_refresher, payload_context) - file_source_validator = FileSourceValidator() - search_client = SearchClient(self.search_service_url, token_refresher, payload_context) - source_file_checker = SourceFileChecker() - - referential_integrity_validator = ManifestIntegrity( - self.search_service_url, - token_refresher, - file_source_validator, - payload_context - ) - - manifest_processor = ManifestProcessor( - storage_url=self.storage_url, - file_handler=file_handler, - token_refresher=token_refresher, - context=payload_context, - source_file_checker=source_file_checker, - ) - validator = SchemaValidator( - self.schema_service_url, - token_refresher, - payload_context, - data_types_with_surrogate_ids=DATA_TYPES_WITH_SURROGATE_KEYS, - surrogate_key_fields_paths=SURROGATE_KEYS_PATHS - ) - single_manifest_processor = SingleManifestProcessor( - storage_url=self.storage_url, - payload_context=payload_context, - referential_integrity_validator=referential_integrity_validator, - manifest_processor=manifest_processor, - schema_validator=validator, - token_refresher=token_refresher, - ) - - manifest_data = self._get_manifest_data(context, execution_context) - logger.debug(f"Manifest data: {manifest_data}") - - if not manifest_data: - raise EmptyManifestError( - f"Data {context['dag_run'].conf} doesn't contain 'manifest field'") - record_ids, skipped_ids = self._process_manifest(single_manifest_processor, manifest_data) - logger.info(f"Processed ids {record_ids}") - context["ti"].xcom_push(key="record_ids", value=record_ids) - if self._show_skipped_ids: - context["ti"].xcom_push(key="skipped_ids", value=skipped_ids) diff --git a/src/osdu_dags/osdu_manifest/operators/search_record_id.py b/src/osdu_dags/osdu_manifest/operators/search_record_id.py deleted file mode 100644 index 04a9d658a88fd6839f7eb9f2e3edda566cb6f740..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/search_record_id.py +++ /dev/null @@ -1,50 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - -import logging - -from airflow.models import BaseOperator, Variable -from osdu_api.libs.context import Context -from osdu_api.libs.refresh_token import AirflowTokenRefresher -from osdu_api.libs.search_record_ids import SearchId - -logger = logging.getLogger() - - -class SearchRecordIdOperator(BaseOperator): - """Operator to search files in SearchService by record ids. - Expects "record_ids" field in xcom. - """ - ui_color = '#10ECAA' - ui_fgcolor = '#000000' - - FINISHED_STATUS = "finished" - RUNNING_STATUS = "running" - FAILED_STATUS = "failed" - - def execute(self, context: dict): - """Execute update workflow status. - If status assumed to be FINISHED then we check whether proceed files - are searchable or not. - If they are then update status FINISHED else FAILED - - :param context: Airflow dagrun context - :type context: dict - """ - payload_context = Context.populate(context["dag_run"].conf) - record_ids = context["ti"].xcom_pull(key="record_ids", ) - ids_searcher = SearchId(Variable.get("core__service__search__url"), record_ids, AirflowTokenRefresher(), - payload_context) - ids_searcher.check_records_searchable() diff --git a/src/osdu_dags/osdu_manifest/operators/update_status.py b/src/osdu_dags/osdu_manifest/operators/update_status.py deleted file mode 100644 index fbfca1b15a6789627ac52e1a5bed11df43912fd5..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/update_status.py +++ /dev/null @@ -1,133 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - -"""Update Status operator.""" - -import copy -import enum -import logging -from typing import Tuple - -from airflow.models import BaseOperator, Variable -from osdu_airflow.backward_compatibility.airflow_utils import apply_defaults -from osdu_api.libs.context import Context -from osdu_api.libs.exceptions import PipelineFailedError -from osdu_api.libs.refresh_token import AirflowTokenRefresher -from osdu_api.libs.update_status import UpdateStatus - -logger = logging.getLogger() - - -class UpdateStatusOperator(BaseOperator): - """Operator to update status.""" - ui_color = '#10ECAA' - ui_fgcolor = '#000000' - - @apply_defaults - def __init__(self, *args, **kwargs): - """Init base operator and obtain base urls from Airflow Variables.""" - super().__init__(*args, **kwargs) - self._show_skipped_ids = Variable.get('core__config__show_skipped_ids', default_var=False) - - class prev_ti_state(enum.Enum): - NONE = "running" - SUCCESS = "finished" - FAILED = "failed" - - def get_previous_ti_statuses(self, context: dict) -> enum.Enum: - """Get status of previous tasks' executions. - Return corresponding enum value. - - :param context: Airflow context - :type context: dict - :return: Previous status - :rtype: enum.Enum - """ - dagrun = context['ti'].get_dagrun() - failed_ti = dagrun.get_task_instances(state='failed') - success_ti = dagrun.get_task_instances(state='success') - if not failed_ti and not success_ti: # There is no prev task so it can't have been failed - logger.info("There are no tasks before this one. So it has status RUNNING") - return self.prev_ti_state.NONE - if failed_ti: - logger.info("There are failed tasks before this one. So it has status FAILED") - return self.prev_ti_state.FAILED - logger.info("There are successed tasks before this one. So it has status SUCCESSED") - return self.prev_ti_state.SUCCESS - - def pre_execute(self, context: dict): - self.status = self.get_previous_ti_statuses(context) - - def _create_skipped_report(self, context: dict) -> Tuple[dict, dict]: - """ - Return aggregated report of skipped ids grouoped by tasks - - :param context: - :return: Aggregated report grouped by tasks - """ - skipped_ids_report = {} - saved_record_ids = {} - dagrun = context['ti'].get_dagrun() - task_instances = dagrun.get_task_instances() - for task in task_instances: - task_skipped_ids = context["ti"].xcom_pull(key="skipped_ids", task_ids=task.task_id) - if task_skipped_ids: - skipped_ids_report[task.task_id] = task_skipped_ids - - for task in task_instances: - task_saved_ids = context["ti"].xcom_pull(key="record_ids", task_ids=task.task_id) - if task_saved_ids: - saved_record_ids[task.task_id] = task_saved_ids - return skipped_ids_report, saved_record_ids - - def execute(self, context: dict): - """Execute update workflow status. - If status assumed to be FINISHED then we check whether records - are searchable or not. - If they are then update status FINISHED else FAILED - - :param context: Airflow context - :type context: dict - :raises PipelineFailedError: If any of the status is failed - """ - conf = copy.deepcopy(context["dag_run"].conf) - logger.debug(f"Got conf {conf}.") - execution_context = conf["execution_context"] - if "Payload" in execution_context: - payload_context = Context.populate(execution_context) - else: - payload_context = Context(data_partition_id=execution_context["data-partition-id"], - app_key=execution_context.get("AppKey", "")) - workflow_name = conf["workflow_name"] - run_id = conf["run_id"] - status = self.status.value - status_updater = UpdateStatus( - workflow_name=workflow_name, - workflow_url=Variable.get("core__service__workflow__host"), - workflow_id="", - run_id=run_id, - status=status, - token_refresher=AirflowTokenRefresher(), - context=payload_context - ) - status_updater.update_workflow_status() - - if self._show_skipped_ids: - skipped_ids, saved_record_ids = self._create_skipped_report(context) - context["ti"].xcom_push(key="skipped_ids", value=skipped_ids) - context["ti"].xcom_push(key="saved_record_ids", value=saved_record_ids) - - if self.status is self.prev_ti_state.FAILED: - raise PipelineFailedError("Dag failed") diff --git a/src/osdu_dags/osdu_manifest/operators/validate_manifest_schema.py b/src/osdu_dags/osdu_manifest/operators/validate_manifest_schema.py deleted file mode 100644 index 902aa5d4a3356907e949939440875245be73c652..0000000000000000000000000000000000000000 --- a/src/osdu_dags/osdu_manifest/operators/validate_manifest_schema.py +++ /dev/null @@ -1,89 +0,0 @@ -# Copyright 2021 Google LLC -# Copyright 2021 EPAM Systems -# -# Licensed 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. - -""" -Validate Manifest against R3 schemas operator. -""" - -import logging - -from airflow.models import BaseOperator, Variable -from osdu_airflow.backward_compatibility.airflow_utils import apply_defaults -from osdu_api.libs.constants import DATA_TYPES_WITH_SURROGATE_KEYS, SURROGATE_KEYS_PATHS -from osdu_api.libs.context import Context -from osdu_api.libs.exceptions import EmptyManifestError, GenericManifestSchemaError -from osdu_api.libs.refresh_token import AirflowTokenRefresher -from osdu_api.libs.validation.validate_schema import SchemaValidator -from osdu_manifest.operators.mixins.ReceivingContextMixin import ReceivingContextMixin - -logger = logging.getLogger() - - -class ValidateManifestSchemaOperator(BaseOperator, ReceivingContextMixin): - """Operator to validate manifest against definition schemasR3.""" - - ui_color = '#dad5ff' - ui_fgcolor = '#000000' - - @apply_defaults - def __init__(self, previous_task_id: str = None, *args, **kwargs): - """Init base operator and obtain base urls from Airflow Variables.""" - super().__init__(*args, **kwargs) - self.previous_task_id = previous_task_id - self.schema_service_url = Variable.get('core__service__schema__url') - self._show_skipped_ids = Variable.get( - 'core__config__show_skipped_ids', default_var=False - ) - - def execute(self, context: dict): - """Execute manifest validation then process it. - - Get a single manifest file or a list of them. - If it is a list, calculate which range (slice) of manifest files must be processed and then - process this range one by one. - - :param context: Airflow context - :type context: dict - """ - execution_context = context["dag_run"].conf["execution_context"] - payload_context = Context.populate(execution_context) - token_refresher = AirflowTokenRefresher() - - schema_validator = SchemaValidator( - self.schema_service_url, - token_refresher, - payload_context, - surrogate_key_fields_paths=SURROGATE_KEYS_PATHS, - data_types_with_surrogate_ids=DATA_TYPES_WITH_SURROGATE_KEYS - ) - - manifest_data = self._get_manifest_data(context, execution_context) - logger.debug(f"Manifest data: {manifest_data}") - - if not manifest_data: - raise EmptyManifestError( - f"Data {context['dag_run'].conf} doesn't contain 'manifest field'") - - _ = schema_validator.validate_common_schema(manifest_data) - try: - valid_manifest_file, skipped_entities = schema_validator.ensure_manifest_validity( - manifest_data - ) - except GenericManifestSchemaError as err: - context["ti"].xcom_push(key="skipped_ids", value=str(err)) - raise err - if self._show_skipped_ids: - context["ti"].xcom_push(key="skipped_ids", value=skipped_entities) - return {"manifest": valid_manifest_file} diff --git a/tests/plugin-unit-tests/README.md b/tests/plugin-unit-tests/README.md deleted file mode 100644 index dc225cb727844e11cc3bb880026335f6edd6520e..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/README.md +++ /dev/null @@ -1,5 +0,0 @@ -``` -pip install pytest -export AIRFLOW_SRC_DIR=/path/to/airflow-folder -pytest -``` diff --git a/tests/plugin-unit-tests/__init__.py b/tests/plugin-unit-tests/__init__.py deleted file mode 100644 index 022550cb1b433f99dceacfaf9b94e5712ebc8a11..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/__init__.py +++ /dev/null @@ -1,15 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - diff --git a/tests/plugin-unit-tests/data/__init__.py b/tests/plugin-unit-tests/data/__init__.py deleted file mode 100644 index 7be2c556ed3b00241e3a46b707419523a803dcbd..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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 .process_manifest_r2_op import * diff --git a/tests/plugin-unit-tests/data/manifests/schema_Manifest.1.0.0.json b/tests/plugin-unit-tests/data/manifests/schema_Manifest.1.0.0.json deleted file mode 100644 index facd766ba5bac7cddeb2b7534fa0468f5d2a2477..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/manifests/schema_Manifest.1.0.0.json +++ /dev/null @@ -1,55 +0,0 @@ -{ - "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed 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.", - "$id": "https://schema.osdu.opengroup.org/json/manifest/Manifest.1.0.0.json", - "$schema": "http://json-schema.org/draft-07/schema#", - "title": "Load Manifest Schema", - "description": "Load manifest applicable for all types defined as 'kind', i.e. registered as schemas with the Schema Service. It supports loading of individual 'records' of any group-type or combinations. The load sequence follows a well-defined sequence. The 'ReferenceData' array is processed first (if populated). The 'MasterData' array is processed second (if populated) second. The 'Data' structure is processed last (if populated). Inside the 'Data' property the 'Files' array is processed first, followed by the 'WorkProductComponents' array, the 'WorkProduct' is processed last. Any arrays are ordered. should there be interdependencies, the dependent items must be placed behind their relationship targets, e.g. a master-data Well record must placed in the 'MasterData' array before its Wellbores.", - "type": "object", - "properties": { - "kind": { - "description": "The schema identification for the manifest record following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.", - "title": "Manifest Kind", - "type": "string", - "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.\\/]+:[0-9]+.[0-9]+.[0-9]+$", - "example": "osdu:wks:Manifest:1.0.0" - }, - "ReferenceData": { - "description": "Reference-data are submitted as an array of records.", - "type": "array", - "items": { - "$ref": "GenericReferenceData.1.0.0.json" - } - }, - "MasterData": { - "description": "Master-data are submitted as an array of records.", - "type": "array", - "items": { - "$ref": "GenericMasterData.1.0.0.json" - } - }, - "Data": { - "description": "Manifest schema for work-product, work-product-component, dataset ensembles. The items in 'Datasets' are processed first since they are referenced by 'WorkProductComponents' ('data.DatasetIDs[]' and 'data.Artefacts[].ResourceID'). The WorkProduct is processed last collecting the WorkProductComponents.", - "type": "object", - "properties": { - "WorkProduct": { - "description": "The work-product component capturing the work-product-component records belonging to this loading/ingestion transaction.", - "$ref": "GenericWorkProduct.1.0.0.json" - }, - "WorkProductComponents": { - "description": "The list of work-product-components records. The record ids are internal surrogate keys enabling the association of work-product-component records with the work-product records.", - "type": "array", - "items": { - "$ref": "GenericWorkProductComponent.1.0.0.json" - } - }, - "Datasets": { - "description": "The list of 'Datasets' or data containers holding the actual data. The record ids are usually internal surrogate keys enabling the association of dataset records with work-product-component records, namely via 'DatasetIDs' and 'Artefacts.ResourceID' (both referring to 'dataset' group-type entity types).", - "type": "array", - "items": { - "$ref": "GenericDataset.1.0.0.json" - } - } - } - } - } -} diff --git a/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json deleted file mode 100644 index f716b42e92c5a64f933367ea6fbacf0947c3ada6..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json +++ /dev/null @@ -1,57 +0,0 @@ -{ - "execution_context": { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "opendes", - "AppKey": "", - "kind_version": "3.0.0" - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - } - }, - "workflow_name": "osdu_ingest", - "run_id": "foo" -} diff --git a/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json deleted file mode 100644 index b88045eb90fc2b6f5813f70ce508535592ab4cbc..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json +++ /dev/null @@ -1,1177 +0,0 @@ -{ - "execution_context": { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "opendes", - "AppKey": "", - "kind_version": "3.0.0" - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - }, - { - "kind": "test:test:Manifest:1.0.0", - "ReferenceData": [], - "MasterData": [ - { - "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:TestMaster:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "SequenceNumber": 1 - }, - "schema": "test:test:GenericMasterData:1.0.0" - } - ], - "Data": {} - } - ] - }, - "workflow_name": "osdu_ingest", - "run_id": "foo" -} diff --git a/tests/plugin-unit-tests/data/other/SearchResponseValid.json b/tests/plugin-unit-tests/data/other/SearchResponseValid.json deleted file mode 100644 index a8303d36fec5cbea37b27a7e089acb82b3563b25..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/other/SearchResponseValid.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "results": [ - { - "id": "some_test_id", - "version": 12345 - } - ], - "aggregations": null, - "totalCount": 45 -} diff --git a/tests/plugin-unit-tests/data/process_manifest_r2_op.py b/tests/plugin-unit-tests/data/process_manifest_r2_op.py deleted file mode 100644 index fa248ae77fc691bc5e5ff6ae8a92da1ef35ead25..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/process_manifest_r2_op.py +++ /dev/null @@ -1,122 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - - -ACL_DICT = {'viewers': ['data.default.viewers@odes.osdu.test.net'],'owners': ['data.default.owners@odes.osdu.test.net']} -LEGAL_DICT = {'legaltags': ['odes-demo-legaltag'], 'otherRelevantDataCountries': ['FR', 'US', 'CA'],'status': 'compliant'} - -CONF = { - "WorkProduct": { - "ResourceTypeID": "srn:type:work-product/WellLog:", - "ResourceSecurityClassification": "srn:reference-data/ResourceSecurityClassification:RESTRICTED:", - "Data": { - "GroupTypeProperties": { - "Components": [] - }, - "IndividualTypeProperties": { - "Name": "Test AKM LOG 111", - "Description": "Well Log" - }, - "ExtensionProperties": {} - }, - "ComponentsAssociativeIDs": [ - "wpc-1" - ] - }, - "WorkProductComponents": [ - { - "ResourceTypeID": "srn:type:work-product-component/WellLog:", - "ResourceSecurityClassification": "srn:reference-data/ResourceSecurityClassification:RESTRICTED:", - "Data": { - "GroupTypeProperties": { - "Files": [], - "Artefacts": [] - }, - "AssociativeID": "wpc-1", - "FileAssociativeIDs": [ - "f-1" - ] - } - } - ], - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "test", - "AppKey": "test", - "kind_version": "3.0.0", - "acl": { - "viewers": ["data.default.viewers@odes.osdu.joonix.net"], - "owners": ["data.default.owners@odes.osdu.joonix.net"]}, - "legal": { - "legaltags": ["odes-demo-legaltag"], - "otherRelevantDataCountries": ["FR", "US", "CA"]} - }, - "Files": [ - { - "ResourceTypeID": "srn:type:file/las2:", - "ResourceSecurityClassification": "srn:reference-data/ResourceSecurityClassification:RESTRICTED:", - "Data": { - "GroupTypeProperties": { - "FileSource": "", - "PreLoadFilePath": "foo" - }, - "IndividualTypeProperties": {}, - "ExtensionProperties": {} - }, - "AssociativeID": "f-1" - } - ], - "WorkflowID": "foo" -} - -PROCESS_FILE_ITEMS_RESULT = ( - [ - ( - { - 'kind': 'test:osdu:file:3.0.0', - 'legal': {'legaltags': ['odes-demo-legaltag'], 'otherRelevantDataCountries': ['US'], 'status': 'compliant'}, - 'acl': {'viewers': ['data.default.viewers@odes.osdu.test.net'], - 'owners': ['data.default.owners@odes.osdu.test.net']}, - 'data': { - 'ResourceTypeID': 'srn:type:file/las2:', - 'ResourceSecurityClassification': 'srn:reference-data/ResourceSecurityClassification:RESTRICTED:', - 'Data': {'GroupTypeProperties': {'FileSource': '', 'PreLoadFilePath': 'foo'}, 'IndividualTypeProperties': {}, 'ExtensionProperties': {}}, - 'AssociativeID': 'f-1', - 'ResourceID': "" - } - }, - 'File' - ) - ], - ['srn:file/las2:434064998475386:'] -) - -LOADED_CONF = { - "acl": ACL_DICT, - "legal_tag": LEGAL_DICT, - "data_object": CONF - } - -CONF_PAYLOAD = CONF["Payload"] - - -class DAG_RUN: - def __init__(self): - self.conf = CONF - - -DAG_RUN_CONF = { - "dag_run": DAG_RUN() -} diff --git a/tests/plugin-unit-tests/data/process_manifest_r3.py b/tests/plugin-unit-tests/data/process_manifest_r3.py deleted file mode 100644 index eaca093eb3d077c6aa8d0d1c25fafbe3abb01860..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/data/process_manifest_r3.py +++ /dev/null @@ -1,839 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - - -ACL_DICT = {'viewers': ['data.default.viewers@odes.osdu.test.net'], - 'owners': ['data.default.owners@odes.osdu.test.net']} -LEGAL_DICT = {'legaltags': ['odes-demo-legaltag'], 'otherRelevantDataCountries': ['FR', 'US', 'CA'], - 'status': 'compliant'} - -CONF_LOAD_WELL_13104 = { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "opendes", - "AppKey": "", - "kind_version": "3.0.0" - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - { - - "id": "srn:opendes:master-data/Well:131041", - "version": 1, - "kind": "opendes:osdu:Well:0.3.0", - "groupType": "master-data", - "acl": { - "owners": [ - "ownergroup@testcompany.com" - ], - "viewers": [ - "viewgroup@testcompany.com" - ] - }, - "legal": { - "legaltags": [ - "legaltag" - ], - "otherRelevantDataCountries": [ - "GB" - ] - }, - "resourceObjectCreationDateTime": "2012-03-19T07:22Z", - "resourceVersionCreationDateTime": "2012-03-19T07:22Z", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:Public:1", - "data": { - "FacilityTypeID": "srn:opendes:reference-data/FacilityType:WELL:1", - "FacilityOperator": [ - { - "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CHRYSAOR PRODUCTION (U.K.) LIMITED:1" - } - ], - "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:UK_OGA:1", - "SpatialLocation": [ - { - "Wgs84Coordinates": { - "type": "FeatureCollection", - "features": [ - { - "type": "Feature", - "geometry": { - "type": "Point", - "coordinates": [ - 1.896235806, - 53.72433018 - ] - } - } - ] - } - } - ], - "FacilityName": "48/10b-N2", - "FacilityNameAlias": [ - { - "AliasName": "48/10b-N2", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:WELL_NAME:1" - } - ], - "FacilityState": [ - { - "FacilityStateTypeID": "srn:opendes:reference-data/FacilityStateType:PLUGGED:1" - } - ], - "FacilityEvent": [ - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD_DATE:1", - "EffectiveDateTime": "2005-08-14T00:00:00" - } - ], - "VerticalMeasurements": [ - { - "VerticalMeasurementID": "RT", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:DEPTH_DATUM_ELEV:1", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:M:1" - } - ] - } - - } - ] -} - - -CONF_CONTRACTOR_TYPE = { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "opendes", - "AppKey": "", - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "manifest": [ - { - "id": "srn:opendes:reference-data/ContractorType:1", - "kind": "opendes:osdu:ContractorType:1.0.0", - "groupType": "reference-data", - "version": 1, - "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"], - "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]}, - "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"], - "status": "compliant"}, - "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1", - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:1" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:1", - "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:1", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:1", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:1", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:1", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:1", - "data": { - "Name": "Recording", - "Description": "Performs data acquistion", - "Code": "Record" - } - }, - { - "id": "srn:opendes:reference-data/ContractorType:1", - "kind": "opendes:osdu:ContractorType:1.0.0", - "groupType": "reference-data", - "version": 1, - "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"], - "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]}, - "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"], - "status": "compliant"}, - "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1", - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:1" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:1", - "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:1", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:1", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:1", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:1", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:1", - "data": { - "Name": "Line Clearing", - "Description": "Prepares onshore swath access", - "Code": "LineClear" - } - }, - { - "id": "srn:opendes:reference-data/ContractorType:1", - "kind": "opendes:osdu:ContractorType:1.0.0", - "groupType": "reference-data", - "version": 1, - "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"], - "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]}, - "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"], - "status": "compliant"}, - "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1", - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:1" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:1", - "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:1", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:1", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:1", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:1", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:1", - "data": { - "Name": "Positioning", - "Description": "Establishes location of surface equipment", - "Code": "Position" - } - }, - { - "id": "srn:opendes:reference-data/ContractorType:1", - "kind": "opendes:osdu:ContractorType:1.0.0", - "groupType": "reference-data", - "version": 1, - "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"], - "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]}, - "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"], - "status": "compliant"}, - "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1", - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:1" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:", - "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "Name": "Data Processing", - "Description": "Transforms data", - "Code": "DataProc" - } - } - ] -} - - -CONF = { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "opendes", - "AppKey": "", - "kind_version": "3.0.0" - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - { - "id": "srn:opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:Wellbore:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "status": "srn:opendes:reference-data/LegalStatus:public:1111" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "FacilityTypeID": "srn:opendes:reference-data/FacilityType:Wellbore:", - "FacilityOperator": [ - { - "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:" - } - ], - "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "SpatialLocation": [ - { - "Coordinates": [ - { - "x": -98.580887, - "y": 35.6381829999999 - } - ], - "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Point:", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:", - "HorizontalCRSID": "srn:opendes:reference-data/HorizontalCRS:NAD27:", - "HeightAboveGroundLevelUOMID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:" - } - ], - "OperatingEnvironmentID": "srn:opendes:reference-data/OperatingEnvironment:onshore:", - "FacilityName": "IRETA 1-4-9XH", - "FacilityNameAlias": [ - { - "AliasName": " IRETA 1-4-9XH", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:Name:" - }, - { - "AliasName": "350112350400", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:UWBI:" - } - ], - "FacilityEvent": [ - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD:", - "EffectiveDateTime": "2015-03-11T00:00:00-05:00" - }, - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:", - "EffectiveDateTime": "2015-05-18T00:00:00-06:00" - } - ], - "WellID": "srn:opendes:master-data/Well:3501123504:", - "SequenceNumber": 1, - "VerticalMeasurements": [ - { - "VerticalMeasurementID": "TD_1", - "VerticalMeasurement": 0, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Measured Depth:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalReferenceID": "Drill Floor" - }, - { - "VerticalMeasurementID": "TD_2", - "VerticalMeasurement": 0, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:True Vertical Depth:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalReferenceID": "Drill Floor" - }, - { - "VerticalMeasurementID": "Elev_1", - "VerticalMeasurement": 1636, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Drill Floor:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:" - }, - { - "VerticalMeasurementID": "Elev_2", - "VerticalMeasurement": 1606, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Ground Level:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:" - } - ], - "TrajectoryTypeID": "srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:", - "DefaultVerticalMeasurementID": "", - "GeographicBottomHoleLocation": { - "Coordinates": [ - { - "x": -98.580887, - "y": 35.6381829999999 - } - ] - } - } - } - - ], - "WorkflowID": "foo" -} - -CONF_TEST_REFERENCE = { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "opendes", - "AppKey": "", - "kind_version": "3.0.0" - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - - { - "kind": "opendes:osdu:TestReference:1.0.1", - "groupType": "reference-data", - "acl": { - "owners": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "viewers": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "legaltags": [ - "opendes-demo-legaltag" - ], - "otherRelevantDataCountries": [ - "US" - ], - "status": "compliant" - }, - "data": { - "Name": "TestReference", - "Description": "A meaningful description of this TestReference.", - "Schema": "http://json-schema.org/draft-07/schema#", - "SchemaID": "https://schema.osdu.opengroup.org/json/reference-data/TestReference.1.0.0.json", - "SchemaKind": "osdu:osdu:TestReference:1.0.0", - "GroupType": "reference-data", - "IsReferenceValueType": True, - "GovernanceAuthorities": [ - "$$srn:NAMESPACE$$:reference-data/OrganisationType:osdu" - ], - "NaturalKeys": [ - "data.Code", - "data.Name" - ], - "GovernanceModel": "LOCAL" - } - } - - ], - "WorkflowID": "foo" -} - - -CONF2 = { - "WorkflowID": "{{workflow_id}}", - "Payload": { - "AppKey": "", - "data-partition-id": "opendes" - }, - "manifest": [ - { - "id": "srn:opendes:master-data/Wellbore:350112350400", - "kind": "osdu:osdu:Wellbore:0.3.0", - "groupType": "master-data", - "version": 1, - "acl": { - "Viewers": [ - "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ], - "Owners": [ - "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" - ] - }, - "legal": { - "LegalTags": [ - "legaltag1" - ], - "OtherRelevantDataCountries": [ - "srn:opendes:master-data/GeoPoliticalEntity:USA:" - ], - "Status": "srn:opendes:reference-data/LegalStatus:public:" - }, - "resourceHostRegionIDs": [ - "srn:opendes:reference-data/OSDURegion:US-EAST:" - ], - "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", - "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", - "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", - "data": { - "FacilityTypeID": "srn:opendes:reference-data/FacilityType:Wellbore:", - "FacilityOperator": [ - { - "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:" - } - ], - "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "SpatialLocation": [ - { - "Coordinates": [ - { - "x": -98.580887, - "y": 35.6381829999999 - } - ], - "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Point:1", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:1", - "HorizontalCRSID": "srn:opendes:reference-data/HorizontalCRS:NAD27:1", - "HeightAboveGroundLevelUOMID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1" - } - ], - "OperatingEnvironmentID": "srn:opendes:reference-data/OperatingEnvironment:onshore:1", - "FacilityName": "IRETA 1-4-9XH", - "FacilityNameAlias": [ - { - "AliasName": " IRETA 1-4-9XH", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:Name:1" - }, - { - "AliasName": "350112350400", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:UWBI:1" - } - ], - "FacilityEvent": [ - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD:1", - "EffectiveDateTime": "2015-03-11T00:00:00-05:00" - }, - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:1", - "EffectiveDateTime": "2015-05-18T00:00:00-06:00" - } - ], - "WellID": "srn:opendes:master-data/Well:3501123504:1", - "SequenceNumber": 1, - "VerticalMeasurements": [ - { - "VerticalMeasurementID": "TD_1", - "VerticalMeasurement": 0, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:1", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Measured Depth:1", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1", - "VerticalReferenceID": "Drill Floor" - }, - { - "VerticalMeasurementID": "TD_2", - "VerticalMeasurement": 0, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:1", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:True Vertical Depth:1", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1", - "VerticalReferenceID": "Drill Floor" - }, - { - "VerticalMeasurementID": "Elev_1", - "VerticalMeasurement": 1636, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Drill Floor:1", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:1", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:1" - }, - { - "VerticalMeasurementID": "Elev_2", - "VerticalMeasurement": 1606, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Ground Level:1", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:1", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:1" - } - ], - "TrajectoryTypeID": "srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:1", - "DefaultVerticalMeasurementID": "", - "GeographicBottomHoleLocation": { - "Coordinates": [ - { - "x": -98.580887, - "y": 35.6381829999999 - } - ] - } - } - } - ] -} - -TEST_SCHEMA = { - "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed 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.", - "$id": "https://schema.osdu.opengroup.org/json/reference-data/ContractorType.1.0.0.json", - "$schema": "http://json-schema.org/draft-07/schema#", - "title": "Test", - "description": "Test.", - "type": "object", - "properties": { - "id": { - "description": "The SRN which identifies this OSDU resource object without version.", - "title": "Entity ID", - "type": "string", - "pattern": "^srn:opendes:master-data\\/Wellbore:[^:]+$", - "example": "srn:opendes:master-data/Wellbore:2adac27b-5d84-5bcd-89f2-93ee709c06d9" - }, - "kind": { - "description": "The schema identification for the OSDU resource object following the pattern opendes:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.", - "title": "Entity Kind", - "type": "string", - "pattern": "^[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[0-9]+.[0-9]+.[0-9]+$", - "example": "namespace:osdu:Wellbore:2.7.112" - }, - "groupType": { - "description": "The OSDU GroupType assigned to this resource object.", - "title": "Group Type", - "const": "master-data" - }, - "version": { - "description": "The version number of this OSDU resource; set by the framework.", - "title": "Version Number", - "type": "integer", - "format": "int64", - "example": 1831253916104085 - }, - "acl": { - "description": "The access control tags associated with this entity.", - "title": "Access Control List", - "type": "object" - }, - "legal": { - "description": "The entity's legal tags and compliance status.", - "title": "Legal Tags", - "type": "object" - }, - "resourceHomeRegionID": { - "description": "The name of the home [cloud environment] region for this OSDU resource object.", - "title": "Resource Home Region ID", - "type": "string", - "pattern": "^srn:opendes:reference-data\\/OSDURegion:[^:]+:[0-9]*$" - }, - "resourceHostRegionIDs": { - "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.", - "title": "Resource Host Region ID", - "type": "array", - "items": { - "type": "string", - "pattern": "^srn:opendes:reference-data\\/OSDURegion:[^:]+:[0-9]*$" - } - }, - "resourceObjectCreationDateTime": { - "description": "Timestamp of the time at which Version 1 of this OSDU resource object was originated.", - "title": "Resource Object Creation DateTime", - "type": "string", - "format": "date-time" - }, - "resourceVersionCreationDateTime": { - "description": "Timestamp of the time when the current version of this resource entered the OSDU.", - "title": "Resource Version Creation DateTime", - "type": "string", - "format": "date-time" - }, - "resourceCurationStatus": { - "description": "Describes the current Curation status.", - "title": "Resource Curation Status", - "type": "string", - "pattern": "^srn:opendes:reference-data\\/ResourceCurationStatus:[^:]+:[0-9]*$" - }, - "resourceLifecycleStatus": { - "description": "Describes the current Resource Lifecycle status.", - "title": "Resource Lifecycle Status", - "type": "string", - "pattern": "^srn:opendes:reference-data\\/ResourceLifecycleStatus:[^:]+:[0-9]*$" - }, - "resourceSecurityClassification": { - "description": "Classifies the security level of the resource.", - "title": "Resource Security Classification", - "type": "string", - "pattern": "^srn:opendes:reference-data\\/ResourceSecurityClassification:[^:]+:[0-9]*$" - }, - "ancestry": { - "description": "The links to data, which constitute the inputs.", - "title": "Ancestry", - "$ref": "osdu:osdu:AbstractLegalParentList:1.0.0" - }, - "source": { - "description": "Where did the data resource originate? This could be many kinds of entities, such as company, agency, team or individual.", - "title": "Data Source", - "type": "string", - "pattern": "^srn:opendes:master-data\\/Organisation:[^:]+:[0-9]*$" - }, - "existenceKind": { - "description": "Where does this data resource sit in the cradle-to-grave span of its existence?", - "title": "Existence Kind", - "type": "string", - "pattern": "^srn:opendes:reference-data\\/ExistenceKind:[^:]+:[0-9]*$" - }, - "licenseState": { - "description": "Indicates what kind of ownership Company has over data.", - "title": "License State", - "type": "string", - "pattern": "^srn:opendes:reference-data\\/LicenseState:[^:]+:[0-9]*$" - }, - "data": { - "allOf": [ - { - "type": "object", - "properties": { - "WellID": { - "type": "string" - } - } - } - ] - }, - "required": [ - "kind", - "acl", - "groupType", - "legal" - ], - "additionalProperties": False - } -} - -EMPTY_MANIFEST = { - "Payload": { - "authorization": "Bearer test", - "data-partition-id": "osdu", - "AppKey": "", - "kind_version": "3.0.0" - }, - "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", - "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [], - "WorkflowID": "foo" -} - -EXPECTED_RECORD = [{'legal': {'LegalTags': ['legaltag1'], 'OtherRelevantDataCountries': [ - 'srn:opendes:master-data/GeoPoliticalEntity:USA:'], - 'Status': 'srn:opendes:reference-data/LegalStatus:public:'}, - 'acl': {'Owners': ['users@odes.osdu.joonix.net'], - 'Viewers': ['users@odes.osdu.joonix.net']}, - 'kind': 'osdu:osdu:Wellbore:0.3.0', 'id': '', - 'data': {'id': 'srn:opendes:master-data/Wellbore:350112350400', - 'groupType': 'master-data', 'version': 1, 'resourceHostRegionIDs': [ - 'srn:opendes:reference-data/OSDURegion:US-EAST:'], - 'resourceObjectCreationDateTime': '2020-10-16T11:14:45-05:00', - 'resourceVersionCreationDateTime': '2020-10-16T11:14:45-05:00', - 'resourceSecurityClassification': 'srn:opendes:reference-data/ResourceSecurityClassification:public:', - 'source': 'srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:', - 'existenceKind': 'srn:opendes:reference-data/ExistenceKind:Active:', - 'licenseState': 'srn:opendes:reference-data/LicenseState:Unlicensed:', - 'data': { - 'FacilityTypeID': 'srn:opendes:reference-data/FacilityType:Wellbore:', - 'FacilityOperator': [{ - 'FacilityOperatorOrganisationID': 'srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:'}], - 'DataSourceOrganisationID': 'srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:', - 'SpatialLocation': [ - {'Coordinates': [{'x': -98.580887, 'y': 35.6381829999999}], - 'SpatialGeometryTypeID': 'srn:opendes:reference-data/SpatialGeometryType:Point:', - 'VerticalCRSID': 'srn:opendes:reference-data/VerticalCRS:MSL:', - 'HorizontalCRSID': 'srn:opendes:reference-data/HorizontalCRS:NAD27:', - 'HeightAboveGroundLevelUOMID': 'srn:opendes:reference-data/UnitOfMeasure:ft[US]:'}], - 'OperatingEnvironmentID': 'srn:opendes:reference-data/OperatingEnvironment:onshore:', - 'FacilityName': 'IRETA 1-4-9XH', 'FacilityNameAlias': [ - {'AliasName': ' IRETA 1-4-9XH', - 'AliasNameTypeID': 'srn:opendes:reference-data/AliasNameType:Name:'}, - {'AliasName': '350112350400', - 'AliasNameTypeID': 'srn:opendes:reference-data/AliasNameType:UWBI:'}], - 'FacilityEvent': [{ - 'FacilityEventTypeID': 'srn:opendes:reference-data/FacilityEventType:SPUD:', - 'EffectiveDateTime': '2015-03-11T00:00:00-05:00'}, - { - 'FacilityEventTypeID': 'srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:', - 'EffectiveDateTime': '2015-05-18T00:00:00-06:00'}], - 'WellID': 'srn:opendes:master-data/Well:3501123504:', - 'SequenceNumber': 1, 'VerticalMeasurements': [ - {'VerticalMeasurementID': 'TD_1', 'VerticalMeasurement': 0, - 'VerticalMeasurementTypeID': 'srn:opendes:reference-data/VerticalMeasurementType:Total Depth:', - 'VerticalMeasurementPathID': 'srn:opendes:reference-data/VerticalMeasurementPath:Measured Depth:', - 'VerticalMeasurementUnitOfMeasureID': 'srn:opendes:reference-data/UnitOfMeasure:ft[US]:', - 'VerticalReferenceID': 'Drill Floor'}, - {'VerticalMeasurementID': 'TD_2', 'VerticalMeasurement': 0, - 'VerticalMeasurementTypeID': 'srn:opendes:reference-data/VerticalMeasurementType:Total Depth:', - 'VerticalMeasurementPathID': 'srn:opendes:reference-data/VerticalMeasurementPath:True Vertical Depth:', - 'VerticalMeasurementUnitOfMeasureID': 'srn:opendes:reference-data/UnitOfMeasure:ft[US]:', - 'VerticalReferenceID': 'Drill Floor'}, - {'VerticalMeasurementID': 'Elev_1', - 'VerticalMeasurement': 1636, - 'VerticalMeasurementTypeID': 'srn:opendes:reference-data/VerticalMeasurementType:Drill Floor:', - 'VerticalMeasurementPathID': 'srn:opendes:reference-data/VerticalMeasurementPath:Elevation:', - 'VerticalMeasurementUnitOfMeasureID': 'srn:opendes:reference-data/UnitOfMeasure:ft[US]:', - 'VerticalCRSID': 'srn:opendes:reference-data/VerticalCRS:MSL:'}, - {'VerticalMeasurementID': 'Elev_2', - 'VerticalMeasurement': 1606, - 'VerticalMeasurementTypeID': 'srn:opendes:reference-data/VerticalMeasurementType:Ground Level:', - 'VerticalMeasurementPathID': 'srn:opendes:reference-data/VerticalMeasurementPath:Elevation:', - 'VerticalMeasurementUnitOfMeasureID': 'srn:opendes:reference-data/UnitOfMeasure:ft[US]:', - 'VerticalCRSID': 'srn:opendes:reference-data/VerticalCRS:MSL:'}], - 'TrajectoryTypeID': 'srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:', - 'DefaultVerticalMeasurementID': '', - 'GeographicBottomHoleLocation': { - 'Coordinates': [{'x': -98.580887, 'y': 35.6381829999999}]}}}}, - {'legal': {'legaltags': ['$$LEGAL_TAG$$'], - 'otherRelevantDataCountries': ['$$ISO_3166_ALPHA_2_CODE$$']}, - 'acl': {'owners': ['$$DATA_OWNERS_GROUP$$'], - 'viewers': ['$$DATA_VIEWERS_GROUP$$']}, - 'kind': 'osdu:osdu:TestReference:1.0.0', 'id': '', - 'data': {'id': '$$srn:NAMESPACE$$:type/Type:TestReference', 'groupType': 'type', - 'data': {'Name': 'TestReference', - 'Description': 'A meaningful description of this TestReference.', - 'Schema': 'http://json-schema.org/draft-07/schema#', - 'SchemaID': 'https://schema.osdu.opengroup.org/json/reference-data/TestReference.1.0.0.json', - 'SchemaKind': 'osdu:osdu:TestReference:1.0.0', - 'GroupType': 'reference-data', 'IsReferenceValueType': True, - 'GovernanceAuthorities': [ - '$$srn:NAMESPACE$$:reference-data/OrganisationType:osdu'], - 'NaturalKeys': ['data.Code', 'data.Name'], - 'GovernanceModel': 'LOCAL'}}}, { - 'legal': {'legalTags': ['$$LEGAL_TAG$$'], - 'otherRelevantDataCountries': ['$$ISO_3166_ALPHA_2_CODE$$']}, - 'acl': {'owners': ['$$DATA_OWNERS_GROUP$$'], - 'viewers': ['$$DATA_VIEWERS_GROUP$$']}, - 'kind': 'osdu:osdu:UnitQuantity:1.0.0', 'id': '', - 'data': {'id': '$$srn:NAMESPACE$$:reference-data/UnitQuantity:1', - 'groupType': 'reference-data', - 'resourceObjectCreationDateTime': '2020-10-08T12:16:15Z', - 'resourceVersionCreationDateTime': '2020-10-08T12:16:15Z', - 'source': 'Workbook Authoring/UnitQuantity.1.0.0.xlsx; commit SHA 3159b9b1.', - 'Name': 'dimensionless', 'ID': '1', 'InactiveIndicator': False, - 'Code': '1', 'AttributionAuthority': 'Energistics', - 'AttributionPublication': 'Energistics Unit of Measure Dictionary V1.0', - 'AttributionRevision': '1.0', 'BaseForConversion': 'Euc', - 'ParentUnitQuantity': '1', - 'PersistableReference': '{"ancestry":"1","type":"UM"}', - 'UnitDimension': '1'}}] - -PROCESS_FILE_ITEMS_RESULT = ( - [ - ( - { - 'kind': 'test:osdu:file:3.0.0', - 'legal': {'legaltags': ['odes-demo-legaltag'], 'otherRelevantDataCountries': ['US'], - 'status': 'compliant'}, - 'acl': {'viewers': ['data.default.viewers@odes.osdu.test.net'], - 'owners': ['data.default.owners@odes.osdu.test.net']}, - 'data': { - 'ResourceTypeID': 'srn:type:file/las2:', - 'ResourceSecurityClassification': 'srn:reference-data/ResourceSecurityClassification:RESTRICTED:', - 'Data': {'GroupTypeProperties': {'FileSource': '', 'PreLoadFilePath': 'foo'}, - 'IndividualTypeProperties': {}, 'ExtensionProperties': {}}, - 'AssociativeID': 'f-1', - 'ResourceID': "" - } - }, - 'File' - ) - ], - ['srn:file/las2:434064998475386:'] -) - -LOADED_CONF = { - "acl": ACL_DICT, - "legal_tag": LEGAL_DICT, - "data_object": CONF -} - -CONF_PAYLOAD = CONF["Payload"] diff --git a/tests/plugin-unit-tests/file_paths.py b/tests/plugin-unit-tests/file_paths.py deleted file mode 100644 index e9d6aab8f22125dff678558fb6c0a7e7f84e8c23..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/file_paths.py +++ /dev/null @@ -1,25 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - -import os - -DATA_PATH_PREFIX = f"{os.path.dirname(__file__)}/data" - -MANIFEST_GENERIC_SCHEMA_PATH = f"{DATA_PATH_PREFIX}/manifests/schema_Manifest.1.0.0.json" - -MANIFEST_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/Wellbore.0.3.0.json" -MANIFEST_BATCH_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/batch_Wellbore.0.3.0.json" - -SEARCH_VALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseValid.json" diff --git a/tests/plugin-unit-tests/mock_providers.py b/tests/plugin-unit-tests/mock_providers.py deleted file mode 100644 index 7690ff188e33f83fcc397fd29cfe42df85224159..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/mock_providers.py +++ /dev/null @@ -1,118 +0,0 @@ -# Copyright 2021 Google LLC -# Copyright 2021 EPAM Systems -# -# Licensed 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. -"""Mock providers module.""" - -import io -import logging -from typing import Tuple -from osdu_api.providers.blob_storage import get_client -from osdu_api.providers.credentials import get_credentials -from osdu_api.providers.factory import ProvidersFactory -from osdu_api.providers.types import BlobStorageClient, BaseCredentials - -logger = logging.getLogger(__name__) - - -@ProvidersFactory.register("provider_test") -class MockCredentials(BaseCredentials): - """Mock Credentials Provider.""" - - def __init__(self): - self._access_token = "test_token" - - def refresh_token(self) -> str: - """Refresh token. - - :return: Refreshed token - :rtype: str - """ - logger.info("Refreshed token in test.") - - @property - def access_token(self) -> str: - """The access token. - - :return: Access token string. - :rtype: str - """ - return self._access_token - - @access_token.setter - def access_token(self, token: str): - """Set access token - - :param val: The access token - :type val: str - :return: [description] - :rtype: [type] - """ - self._access_token = token - - -@ProvidersFactory.register("provider_test") -class GoogleCloudStorageClient(BlobStorageClient): - """Mock BlobStorage Provider.""" - - def download_to_file(self, uri: str, file: io.BytesIO) -> Tuple[io.BytesIO, str]: - """Download file from the given URI. - - :param uri: The full URI of the file. - :type uri: str - :param file: a file like object - :type file: io.BytesIO - :return: A tuple containing the file and its content-type - :rtype: Tuple[io.BytesIO, str] - """ - pass - - def download_file_as_bytes(self, uri: str) -> Tuple[bytes, str]: - """Download file as bytes from the given URI. - - :param uri: The full URI of the file - :type uri: str - :return: The file as bytes and its content-type - :rtype: Tuple[bytes, str] - """ - pass - - def upload_file(self, uri: str, file: io.BytesIO, content_type: str): - """Upload blob to given URI. - - :param uri: The full target URI of the resource to upload. - :type uri: str - :param file: The file to upload - :type file: FileLikeObject - :param content_type: The content-type of the file to uplaod - :type content_type: str - """ - pass - - def does_file_exist(self, uri: str): - """Verify if a resource exists in the given URI. - - :param uri: The URI of the resource to verify - :type uri: str - """ - pass - - -def get_test_credentials(): - """Utiltiy to get the credentials to use in tests.""" - return get_credentials("provider_test") - - -def get_test_blob_storage_client(): - """Utility to get blob storage client to use in tests.""" - return get_client("provider_test") diff --git a/tests/plugin-unit-tests/mock_responses.py b/tests/plugin-unit-tests/mock_responses.py deleted file mode 100644 index 227f859b9445d3d04b008e7e50f266b0a6dacabb..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/mock_responses.py +++ /dev/null @@ -1,48 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - - -import json -import http -import requests - - -class MockResponse(requests.Response): - """ - Mock response is used for monkey patching requests' methods. - Example usage: monkeypatch.setattr( - requests, "get", lambda *args, **kwargs: MockResponse(http.HTTPStatus.OK) - ) - """ - - def __init__(self, status_code: http.HTTPStatus): - super(MockResponse, self).__init__() - self.status_code = status_code - self.url = "Test" - self.reason = "Test" - - @property - def text(self): - return None - - -class MockWorkflowResponse(MockResponse): - - def __init__(self, json: str = "", status_code: http.HTTPStatus = http.HTTPStatus.OK): - super().__init__(status_code) - self._json = json - - def json(self): - return self._json diff --git a/tests/plugin-unit-tests/test_operators_r3.py b/tests/plugin-unit-tests/test_operators_r3.py deleted file mode 100644 index de4f3e4e3ec74366bbba66dff38dd6dbfb9e5a55..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/test_operators_r3.py +++ /dev/null @@ -1,158 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed 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. - - -import http -import json -import os -import sys -from datetime import datetime -from typing import TypeVar, ClassVar -from airflow import DAG -from airflow.models import TaskInstance - -sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") -sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") - -from osdu_api.libs.exceptions import PipelineFailedError -import pytest -import requests -import mock_providers -from functools import lru_cache - -from file_paths import ( - MANIFEST_WELLBORE_VALID_PATH, - SEARCH_VALID_RESPONSE_PATH, MANIFEST_GENERIC_SCHEMA_PATH, MANIFEST_BATCH_WELLBORE_VALID_PATH) -from operators.process_manifest_r3 import ProcessManifestOperatorR3, SchemaValidator, \ - ManifestProcessor -from operators.update_status import UpdateStatusOperator -from osdu_api.libs.handle_file import FileHandler -from mock_responses import MockWorkflowResponse - -CustomOperator = TypeVar("CustomOperator") - - -class MockDagRun: - def __init__(self, conf): - self.conf = conf - - -class MockStorageResponse(requests.Response): - - def json(self, **kwargs): - return {"recordIds": ["test"]} - - -class TestOperators(object): - - def _create_batch_task(self, operator: ClassVar[CustomOperator]) -> (CustomOperator, dict): - with open(MANIFEST_BATCH_WELLBORE_VALID_PATH) as f: - conf = json.load(f) - dag = DAG(dag_id='batch_osdu_ingest', start_date=datetime.now()) - task: CustomOperator = operator(dag=dag, task_id='anytask') - ti = TaskInstance(task=task, execution_date=datetime.now()) - - context = ti.get_template_context() - context["dag_run"] = MockDagRun(conf) - return task, context - - def _create_task(self, operator: ClassVar[CustomOperator]) -> (CustomOperator, dict): - with open(MANIFEST_WELLBORE_VALID_PATH) as f: - conf = json.load(f) - dag = DAG(dag_id='Osdu_ingest', start_date=datetime.now()) - task: CustomOperator = operator(dag=dag, task_id='anytask') - ti = TaskInstance(task=task, execution_date=datetime.now()) - - context = ti.get_template_context() - context["dag_run"] = MockDagRun(conf) - return task, context - - def test_process_manifest_r3_operator(self, monkeypatch): - - @lru_cache() - def _get_common_schema(*args, **kwargs): - with open(MANIFEST_GENERIC_SCHEMA_PATH) as f: - manifest_schema = json.load(f) - return manifest_schema - - monkeypatch.setattr(SchemaValidator, "get_schema", _get_common_schema) - monkeypatch.setattr(SchemaValidator, "_validate_against_schema", lambda *args, **kwargs: None) - monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj, entities: entities) - monkeypatch.setattr(ManifestProcessor, "save_record_to_storage", - lambda obj, headers, request_data: MockStorageResponse()) - monkeypatch.setattr(FileHandler, "upload_file", - lambda *args, **kwargs: "test") - - task, context = self._create_task(ProcessManifestOperatorR3) - task.pre_execute(context) - task.execute(context) - - def test_process_manifest_r3_operator_batch(self, monkeypatch): - - def _get_common_schema(*args, **kwargs): - with open(MANIFEST_GENERIC_SCHEMA_PATH) as f: - manifest_schema = json.load(f) - return manifest_schema - - monkeypatch.setattr(SchemaValidator, "get_schema", _get_common_schema) - monkeypatch.setattr(SchemaValidator, "_validate_against_schema", lambda *args, **kwargs: None) - monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj, entities: (entities, [])) - monkeypatch.setattr(ManifestProcessor, "save_record_to_storage", - lambda obj, headers, request_data: MockStorageResponse()) - monkeypatch.setattr(FileHandler, "upload_file", - lambda *args, **kwargs: "test") - - task, context = self._create_batch_task(ProcessManifestOperatorR3) - task.pre_execute(context) - task.execute(context) - - def _test_update_status_operator(self, monkeypatch, status: UpdateStatusOperator.prev_ti_state): - monkeypatch.setattr(UpdateStatusOperator, "get_previous_ti_statuses", - lambda obj, context: status) - monkeypatch.setattr(requests, "put", lambda *args, **kwargs: MockWorkflowResponse( - status_code=http.HTTPStatus.OK, json="test")) - - task, context = self._create_task(UpdateStatusOperator) - task.pre_execute(context) - task.execute(context) - - @pytest.mark.parametrize( - "status", - [ - pytest.param( - UpdateStatusOperator.prev_ti_state.NONE - ), - pytest.param( - UpdateStatusOperator.prev_ti_state.SUCCESS - ) - ] - ) - def test_update_status_operator(self, monkeypatch, status): - self._test_update_status_operator(monkeypatch, status) - - @pytest.mark.parametrize( - "status", - [ - pytest.param( - UpdateStatusOperator.prev_ti_state.FAILED - ) - ] - ) - def test_update_status_operator_failed(self, monkeypatch, status): - """ - Test if operator raises PipeLineFailedError if any previous task failed. - """ - with pytest.raises(PipelineFailedError): - self._test_update_status_operator(monkeypatch, status) diff --git a/tests/set_airflow_env.sh b/tests/set_airflow_env.sh index 74904ccb96d671a9847b03f76fdfbfd2d000fc1f..937053889c82c19ddbc4ab393e27637ac106c4dd 100755 --- a/tests/set_airflow_env.sh +++ b/tests/set_airflow_env.sh @@ -25,7 +25,7 @@ pip install azure-keyvault-secrets pip install msal pip install python-keycloak pip install osdu-api==0.10.1.dev0+92014f64 --extra-index-url https://community.opengroup.org/api/v4/projects/148/packages/pypi/simple -pip install osdu-airflow --extra-index-url=https://community.opengroup.org/api/v4/projects/668/packages/pypi/simple +pip install osdu-airflow==0.0.1.dev31+59e58330 --extra-index-url=https://community.opengroup.org/api/v4/projects/668/packages/pypi/simple export WORKFLOW_URL="http://127.0.0.1:5000" export UPDATE_STATUS_URL="http://127.0.0.1:5000/wf/us" export STORAGE_URL="http://127.0.0.1:5000/st"