diff --git a/.gitignore b/.gitignore index 99dae20149e066f1f15960c95e4ed4dc0fb6d03b..740307b738e0c3eef04bd15254a04dc1bb7f78ff 100644 --- a/.gitignore +++ b/.gitignore @@ -34,3 +34,8 @@ # will remove it latter **/schema_registration/ + + +# ignore coverage.py +htmlcov/* +.coverage diff --git a/src/dags/libs/context.py b/src/dags/libs/context.py new file mode 100644 index 0000000000000000000000000000000000000000..c22d2d45b3360b8f80fe400b4a7ce6e9654b4b97 --- /dev/null +++ b/src/dags/libs/context.py @@ -0,0 +1,35 @@ +# 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 dataclasses + + +@dataclasses.dataclass +class Context(object): + """ + Store data-partition-id and AppKey passed via Payload field of dagrun.conf. + """ + data_partition_id: str + app_key: str + + @classmethod + def populate(cls, ctx: dict) -> 'Context': + ctx_payload = ctx.pop('Payload') + ctx_obj = cls( + app_key=ctx_payload['AppKey'], + data_partition_id=ctx_payload['data-partition-id'] + ) + return ctx_obj diff --git a/src/dags/libs/mixins.py b/src/dags/libs/mixins.py new file mode 100644 index 0000000000000000000000000000000000000000..7289a59981f5030867e5bfc89705fe06b96d249a --- /dev/null +++ b/src/dags/libs/mixins.py @@ -0,0 +1,35 @@ +# 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 libs.context import Context + + +class HeadersMixin(object): + """ + Mixin for creating request headers to OSDU services using context. + """ + + def __init__(self, context: Context): + self.context = context + + @property + def request_headers(self) -> dict: + headers = { + 'Content-type': 'application/json', + 'data-partition-id': self.context.data_partition_id, + 'AppKey': self.context.app_key + } + return headers diff --git a/src/dags/libs/process_manifest_r3.py b/src/dags/libs/process_manifest_r3.py new file mode 100644 index 0000000000000000000000000000000000000000..4b2f55b060724b42d4c250268a37bcd909837325 --- /dev/null +++ b/src/dags/libs/process_manifest_r3.py @@ -0,0 +1,177 @@ +# 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 copy +import json +import logging +import uuid +from typing import List + +import requests +import tenacity +from libs.context import Context +from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError +from libs.mixins import HeadersMixin +from libs.refresh_token import AirflowTokenRefresher, refresh_token + +logger = logging.getLogger() + +RETRIES = 3 +TIMEOUT = 1 + + +class ManifestProcessor(HeadersMixin): + """Class to process WP, Master and Reference data""" + RECORD_TEMPLATE = { + "legal": {}, + "acl": {}, + "kind": "", + "id": "", + "data": { + } + } + + def __init__(self, storage_url: str, dagrun_conf: dict, context: Context): + super().__init__(context) + self.storage_url = storage_url + self.data_object = copy.deepcopy(dagrun_conf) + self.context = context + + @staticmethod + def _get_kind_name(kind: str) -> str: + """ + osdu:osdu:Well:1.0.0 -> Well + """ + kind_name = kind.split(":")[2] + return kind_name + + def generate_id(self, manifest_fragment: dict) -> str: + """ + Generate id to use it in Storage. + """ + group_type = manifest_fragment.get("groupType", "doc") + kind = manifest_fragment.get("kind") + kind_name = self._get_kind_name(kind) + _id = f"{self.context.data_partition_id}:{group_type}_{kind_name}:{str(uuid.uuid4())}" + return _id + + def populate_manifest_storage_record(self, manifest: dict) -> dict: + """ + Create a record from Master-manifest to store it in Storage service + """ + record = copy.deepcopy(self.RECORD_TEMPLATE) + record["id"] = manifest["id"] if manifest.get("id") else self.generate_id(manifest) + record["kind"] = manifest.pop("kind") + record["legal"] = manifest.pop("legal") + record["acl"] = manifest.pop("acl") + record["data"] = manifest + return record + + def _validate_storage_response(self, response_dict: dict): + if not ( + isinstance(response_dict, dict) and + isinstance(response_dict.get("recordIds"), list) + ): + raise ValueError(f"Invalid answer from Storage server: {response_dict}") + + @tenacity.retry( + wait=tenacity.wait_fixed(TIMEOUT), + stop=tenacity.stop_after_attempt(RETRIES), + reraise=True + ) + @refresh_token(AirflowTokenRefresher()) + def save_record(self, headers: dict, request_data: List[dict]) -> requests.Response: + """ + Send request to record storage API. + """ + request_data = json.dumps(request_data) + logger.info("Send to Storage service") + logger.info(f"{request_data}") + response = requests.put(self.storage_url, request_data, headers=headers) + if response.ok: + response_dict = response.json() + self._validate_storage_response(response_dict) + logger.info(f"Response: {response_dict}") + logger.info(",".join(map(str, response_dict["recordIds"]))) + else: + reason = response.text[:250] + logger.error(f"Request error.") + logger.error(f"Response status: {response.status_code}. " + f"Response content: {reason}.") + return response + + def process_work_product(self, manifest: dict) -> List[dict]: + """ + Process WP. + """ + wp = manifest["WorkProduct"] + records = [self.populate_manifest_storage_record(wp)] + return records + + def process_work_product_components(self, manifest: dict) -> List[dict]: + """ + Process list of WPS. + """ + records = [] + for wpc in manifest["WorkProductComponents"]: + record = self.populate_manifest_storage_record(wpc) + records.append(record) + return records + + def process_work_product_files(self, manifest: dict) -> List[dict]: + """ + Process list of files. + """ + records = [] + for file in manifest["Files"]: + record = self.populate_manifest_storage_record(file) + records.append(record) + return records + + def process_work_product_manifest(self, manifest: dict) -> List[dict]: + file_records = self.process_work_product_files(manifest) + wpc_records = self.process_work_product_components(manifest) + wp_records = self.process_work_product(manifest) + records = file_records + wpc_records + wp_records + return records + + def create_manifest_records(self) -> List[dict]: + """ + Process every record in manifest field + Return list of records ready to be saved into Storage service. + """ + manifest_records = [] + manifests = self.data_object["manifest"] + for manifest in manifests: + if "WorkProduct" in manifest: + wp_records = self.process_work_product_manifest(manifest) + manifest_records.extend(wp_records) + else: + record = self.populate_manifest_storage_record(manifest) + manifest_records.append(record) + return manifest_records + + def process_manifest(self) -> List[str]: + """ + Process manifests and save them into Storage service. + Returns recordIds of saved records. + """ + if "manifest" in self.data_object: + manifest_records = self.create_manifest_records() + else: + raise EmptyManifestError + response = self.save_record(self.request_headers, request_data=manifest_records) + record_ids = response.json()["recordIds"] + return record_ids diff --git a/src/dags/libs/refresh_token.py b/src/dags/libs/refresh_token.py index a02d6b8ebc48d2d81e27acf51b2651984a6a3a58..6228694a4a84de3072358d9eb1b3bf2e31b9360c 100644 --- a/src/dags/libs/refresh_token.py +++ b/src/dags/libs/refresh_token.py @@ -13,13 +13,12 @@ # See the License for the specific language governing permissions and # limitations under the License. -import enum import json import logging import os -import sys from typing import Callable, Union from abc import ABC, abstractmethod +from functools import partial from http import HTTPStatus from urllib.parse import urlparse @@ -30,13 +29,7 @@ from google.oauth2 import service_account from libs.exceptions import RefreshSATokenError, SAFilePathError from tenacity import retry, stop_after_attempt -# 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) +logger = logging.getLogger() RETRIES = 3 @@ -68,7 +61,7 @@ class TokenRefresher(ABC): class AirflowTokenRefresher(TokenRefresher): DEFAULT_ACCESS_SCOPES = ['openid', 'email', 'profile'] - def __init__(self, access_scopes: list=None): + def __init__(self, access_scopes: list = None): from airflow.models import Variable self.airflow_variables = Variable self._access_token = None @@ -86,7 +79,7 @@ class AirflowTokenRefresher(TokenRefresher): @staticmethod @retry(stop=stop_after_attempt(RETRIES)) - def get_sa_file_content_from_google_storage(bucket_name: str, source_blob_name: str) -> str: + def get_sa_info_from_google_storage(bucket_name: str, source_blob_name: str) -> dict: """ Get sa_file content from Google Storage. """ @@ -94,9 +87,15 @@ class AirflowTokenRefresher(TokenRefresher): bucket = storage_client.bucket(bucket_name) blob = bucket.blob(source_blob_name) logger.info("Got SA_file.") - return blob.download_as_string() + sa_info = json.loads(blob.download_as_string()) + return sa_info - def get_sa_file_info(self) -> dict: + @staticmethod + def get_sa_info_from_file(path: str) -> dict: + with open(path) as f: + return json.load(f) + + def get_sa_info(self) -> dict: """ Get file path from SA_FILE_PATH environmental variable. This path can be GCS object URI or local file path. @@ -107,28 +106,30 @@ class AirflowTokenRefresher(TokenRefresher): if parsed_path.scheme == "gs": bucket_name = parsed_path.netloc source_blob_name = parsed_path.path[1:] # delete the first slash - sa_file_content = self.get_sa_file_content_from_google_storage(bucket_name, - source_blob_name) - sa_file_info = json.loads(sa_file_content) + sa_info = self.get_sa_info_from_google_storage(bucket_name, source_blob_name) elif not parsed_path.scheme and os.path.isfile(parsed_path.path): - with open(parsed_path.path) as f: - sa_file_info = json.load(f) + sa_info = self.get_sa_info_from_file(parsed_path.path) else: - raise SAFilePathError - return sa_file_info + logger.error("SA file path error.") + raise SAFilePathError(f"Got path {os.environ.get('SA_FILE_PATH', None)}") + return sa_info @retry(stop=stop_after_attempt(RETRIES)) - def get_access_token_using_sa_file(self) -> str: - """ - Get new access token using SA info. - """ - sa_file_content = self.get_sa_file_info() + def _get_credentials_from_sa_info(self, sa_info: dict) -> service_account.Credentials: try: credentials = service_account.Credentials.from_service_account_info( - sa_file_content, scopes=self.access_scopes) + sa_info, scopes=self.access_scopes) except ValueError as e: logger.error("SA file has bad format.") raise e + return credentials + + def get_access_token_using_sa_file(self) -> str: + """ + Get new access token using SA info. + """ + sa_info = self.get_sa_info() + credentials = self._get_credentials_from_sa_info(sa_info) logger.info("Refresh token.") credentials.refresh(Request()) @@ -171,9 +172,9 @@ def make_callable_request(obj: Union[object, None], request_function: Callable, Create send_request_with_auth function. """ if obj: # if wrapped function is an object's method - callable_request = lambda: request_function(obj, headers, *args, **kwargs) + callable_request = partial(request_function, obj, headers, *args, **kwargs) else: - callable_request = lambda: request_function(headers, *args, **kwargs) + callable_request = partial(request_function, headers, *args, **kwargs) return callable_request @@ -199,7 +200,7 @@ def _validate_token_refresher_type(token_refresher: TokenRefresher): def send_request_with_auth_header(token_refresher: TokenRefresher, *args, - **kwargs) -> requests.Response: + **kwargs) -> requests.Response: """ Send request with authorization token. If response status is in HTTPStatus.UNAUTHORIZED or HTTPStatus.FORBIDDEN, then refresh token and send request once again. @@ -224,7 +225,11 @@ def send_request_with_auth_header(token_refresher: TokenRefresher, *args, headers, *args, **kwargs) response = send_request_with_auth() - response.raise_for_status() + try: + response.raise_for_status() + except requests.HTTPError as e: + logger.error(f"{response.text}") + raise e return response @@ -238,6 +243,7 @@ def refresh_token(token_refresher: TokenRefresher) -> Callable: Or method: request_method(self, header: dict, *args, **kwargs) -> requests.Response """ + _validate_token_refresher_type(token_refresher) def refresh_token_wrapper(request_function: Callable) -> Callable: @@ -245,17 +251,17 @@ def refresh_token(token_refresher: TokenRefresher) -> Callable: if is_method: def _wrapper(obj: object, headers: dict, *args, **kwargs) -> requests.Response: return send_request_with_auth_header(token_refresher, - request_function=request_function, - obj=obj, - headers=headers, - *args, - **kwargs) + request_function=request_function, + obj=obj, + headers=headers, + *args, + **kwargs) else: def _wrapper(headers: dict, *args, **kwargs) -> requests.Response: return send_request_with_auth_header(token_refresher, - request_function=request_function, - headers=headers, - *args, **kwargs) + request_function=request_function, + headers=headers, + *args, **kwargs) return _wrapper return refresh_token_wrapper diff --git a/src/dags/libs/search_record_ids.py b/src/dags/libs/search_record_ids.py new file mode 100644 index 0000000000000000000000000000000000000000..73859305c24a42fbaff37e5740b92fa125bfcd82 --- /dev/null +++ b/src/dags/libs/search_record_ids.py @@ -0,0 +1,103 @@ +# 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 logging + +import requests +import tenacity +from libs.context import Context +from libs.exceptions import RecordsNotSearchableError +from libs.mixins import HeadersMixin +from libs.refresh_token import AirflowTokenRefresher, refresh_token + +logger = logging.getLogger() + +RETRIES = 5 +WAIT = 5 +TIMEOUT = 10 + + +class SearchId(HeadersMixin): + + def __init__(self, search_url: str, record_ids: list, context: Context): + super().__init__(context) + if not record_ids: + logger.error("There are no record ids") + raise ValueError("There are no record id") + self.record_ids = record_ids + self.search_url = search_url + self.expected_total_count = len(record_ids) + self._create_request_body() + + def _create_search_query(self) -> str: + """ + Create search query to send to Search service using recordIds need to be found. + """ + record_ids = " OR ".join(f"\"{id_}\"" for id_ in self.record_ids) + logger.info(f"Search query {record_ids}") + query = f"id:({record_ids})" + return query + + def _create_request_body(self): + """ + Create request body to send to Search service. + """ + query = self._create_search_query() + request_body = { + "kind": "*:*:*:*", + "query": query + } + self.request_body = json.dumps(request_body) + + def _is_record_searchable(self, response: requests.Response) -> bool: + """ + Check if search service returns expected totalCount of records. + """ + logger.info(response.text) + data = response.json() + total_count = data.get('totalCount') + logger.info(f"Got total count {total_count}") + if total_count is None: + raise ValueError(f"Got no totalCount field in Search service response. " + f"Response is {data}.") + return total_count == self.expected_total_count + + @tenacity.retry( + wait=tenacity.wait_exponential(WAIT), + stop=tenacity.stop_after_attempt(RETRIES), + reraise=True + ) + @refresh_token(AirflowTokenRefresher()) + def search_files(self, headers: dict) -> requests.Response: + """ + Send request with recordIds to Search service. + """ + if self.request_body: + response = requests.post(self.search_url, self.request_body, headers=headers) + if not self._is_record_searchable(response): + logger.error("Expected amount (%s) of records not found." % + self.expected_total_count, + ) + raise RecordsNotSearchableError + return response + + def check_records_searchable(self): + """ + Check if every record in self.record_ids is searchable. + """ + headers = self.request_headers + self.search_files(headers) diff --git a/src/dags/libs/update_status.py b/src/dags/libs/update_status.py new file mode 100644 index 0000000000000000000000000000000000000000..e00ea18e9af2774aceccac9cf1797cdfbc8f1ea8 --- /dev/null +++ b/src/dags/libs/update_status.py @@ -0,0 +1,59 @@ +# 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 logging + +import requests +from libs.context import Context +from libs.mixins import HeadersMixin +from libs.refresh_token import AirflowTokenRefresher, refresh_token + +logger = logging.getLogger() + + +class UpdateStatus(HeadersMixin): + + def __init__( + self, + workflow_id: str, + workflow_url: str, + status: str, + context: Context, + ) -> None: + super().__init__(context) + self.workflow_url = workflow_url + self.workflow_id = workflow_id + self.context = context + self.status = status + + @refresh_token(AirflowTokenRefresher()) + def update_status_request(self, headers: dict) -> requests.Response: + request_body = { + "WorkflowID": self.workflow_id, + "Status": self.status + } + request_body = json.dumps(request_body) + logger.info(f" Sending request '{request_body}'") + response = requests.post(self.workflow_url, request_body, headers=headers) + return response + + def update_workflow_status(self): + """ + Update current workflowID. + """ + headers = self.request_headers + self.update_status_request(headers) diff --git a/src/dags/libs/validate_schema.py b/src/dags/libs/validate_schema.py new file mode 100644 index 0000000000000000000000000000000000000000..99edb682c54cb295c776371dcf95f35868d7e043 --- /dev/null +++ b/src/dags/libs/validate_schema.py @@ -0,0 +1,138 @@ +# 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 copy +import logging + +import jsonschema +import requests +import tenacity +from libs.context import Context +from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError +from libs.mixins import HeadersMixin +from libs.refresh_token import AirflowTokenRefresher, refresh_token + +logger = logging.getLogger() + +RETRIES = 3 +TIMEOUT = 1 + + +class OSDURefResolver(jsonschema.RefResolver): + + def __init__(self, schema_service: str, *args, **kwargs): + super(OSDURefResolver, self).__init__(*args, **kwargs) + self.schema_service = schema_service + + def resolve_fragment(self, document: dict, fragment: str) -> dict: + """ + Extend base resolve_fragment method. If a nested schema has 'definitions' field and there + is a schema under this 'definitions', jsonschema attempts to use the id field of this + double-nested schema as URI to get this schema later. So it has sense to replace this id + with a correct one. + """ + document = super().resolve_fragment(document, fragment) + fragment_parts = fragment.split("/") # /definitions/<OsduID> -> [..., <OsduID>] + if len(fragment_parts) > 1: + osdu_id = fragment_parts[-1] + url = f"{self.schema_service}/{osdu_id}" + document["$id"] = url + return document + + +class SchemaValidator(HeadersMixin): + """Class to validate schema of Manifests.""" + + def __init__(self, schema_service: str, dagrun_conf: dict, context: Context): + super().__init__(context) + self.schema_service = schema_service + self.data_object = copy.deepcopy(dagrun_conf) + self.context = context + self.resolver_handlers = { + "osdu": self.get_schema_request, + "https": self.get_schema_request, + self.context.data_partition_id: self.get_schema_request + } + + @tenacity.retry( + wait=tenacity.wait_fixed(TIMEOUT), + stop=tenacity.stop_after_attempt(RETRIES), + reraise=True + ) + @refresh_token(AirflowTokenRefresher()) + def _get_schema_from_schema_service(self, headers: dict, uri: str) -> requests.Response: + """ + Request to Schema service to retrieve schema. + """ + response = requests.get(uri, headers=headers, timeout=60) + return response + + def get_schema_request(self, uri: str) -> dict: + """ + Get Schema from Schema service. Change $id field to url of getting schema. + """ + if uri.startswith("osdu") or uri.startswith(self.context.data_partition_id): + uri = f"{self.schema_service}/{uri}" + response = self._get_schema_from_schema_service(self.request_headers, uri).json() + response["$id"] = uri + return response + + def get_schema(self, kind: str) -> dict: + manifest_schema_uri = f"{self.schema_service}/{kind}" + try: + response = self.get_schema_request(manifest_schema_uri) + except Exception as e: + logger.error(f"Error on getting schema of kind '{kind}'") + raise e + return response + + def _validate_schema(self, manifest: dict, schema: dict = None): + """ + Validate schema. If argument schema is not defined, then use schema service to retrieve + corresponding schema. + """ + if not schema: + schema = self.get_schema(manifest["kind"]) + logger.info(f"Validating kind {manifest['kind']}") + resolver = OSDURefResolver(schema_service=self.schema_service, + base_uri=schema.get("$id", ""), referrer=schema, + handlers=self.resolver_handlers, cache_remote=True) + validator = jsonschema.Draft7Validator(schema=schema, resolver=resolver) + validator.validate(manifest) + + def validate_work_product(self, work_product: dict): + """ + Validate WP manifest. Raise error if manifest is not valid. + """ + for key, value in work_product.items(): + if key != "WorkProduct": + for component in value: + self._validate_schema(component) + else: + self._validate_schema(value) + + def validate_manifest(self): + """ + Validate manifest. Raise error if manifest is not valid. + """ + if "manifest" not in self.data_object: + raise EmptyManifestError + for manifest in self.data_object["manifest"]: + if isinstance(manifest, dict) and manifest.get("kind"): + self._validate_schema(manifest) + elif manifest.get("WorkProductComponents"): + self.validate_work_product(manifest) + else: + raise NotOSDUShemaFormatError(f"Not valid schema {manifest}") diff --git a/src/dags/osdu-ingest-r2.py b/src/dags/osdu-ingest-r2.py index 27891485f27679423740656e72689dc4f84e05c0..37e1de1c8b5f49ea026000ffb8ab8b0bc0eb0886 100644 --- a/src/dags/osdu-ingest-r2.py +++ b/src/dags/osdu-ingest-r2.py @@ -40,13 +40,11 @@ dag = DAG( update_status_running_op = UpdateStatusOperator( task_id="update_status_running_task", - status=UpdateStatusOperator.RUNNING_STATUS, dag=dag ) update_status_finished_op = UpdateStatusOperator( task_id="update_status_finished_task", - status=UpdateStatusOperator.FINISHED_STATUS, dag=dag, trigger_rule="all_done", ) diff --git a/src/plugins/operators/process_manifest_r2.py b/src/plugins/operators/process_manifest_r2.py index 30fbc4266a72241afcc318644542d0fc1fd07787..9a56bdc9f34d26390aac29ece27b0502810eac2b 100644 --- a/src/plugins/operators/process_manifest_r2.py +++ b/src/plugins/operators/process_manifest_r2.py @@ -26,6 +26,7 @@ from typing import Tuple from urllib.error import HTTPError import requests +import tenacity from airflow.models import BaseOperator, Variable from libs.refresh_token import AirflowTokenRefresher, refresh_token @@ -261,6 +262,11 @@ def create_workproduct_request_data(loaded_conf: dict, product_type: str, wp, wp return data_objects_list +@tenacity.retry( + wait=tenacity.wait_fixed(TIMEOUT), + stop=tenacity.stop_after_attempt(RETRIES), + reraise=True +) @refresh_token(AirflowTokenRefresher()) def send_request(headers, request_data): """ diff --git a/src/plugins/operators/process_manifest_r3.py b/src/plugins/operators/process_manifest_r3.py index b4893655bab755edd85efef7691a7f0ec0f9b4a6..6850e41b363dd0dd59ac41990b4a599fa02484ac 100644 --- a/src/plugins/operators/process_manifest_r3.py +++ b/src/plugins/operators/process_manifest_r3.py @@ -13,291 +13,28 @@ # See the License for the specific language governing permissions and # limitations under the License. -import copy -import dataclasses -import json import logging -import sys -import uuid -from typing import List -import jsonschema -import requests -import tenacity +from airflow.utils import apply_defaults from airflow.models import BaseOperator, Variable -from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError -from libs.refresh_token import AirflowTokenRefresher, refresh_token +from libs.context import Context +from libs.process_manifest_r3 import ManifestProcessor +from libs.validate_schema import SchemaValidator -# 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) + +logger = logging.getLogger() RETRIES = 3 TIMEOUT = 1 -@dataclasses.dataclass -class Context(object): - """ - Store data-partition-id and AppKey passed via Payload field of dagrun.conf. - Remove Payload from dagrun.conf. - """ - data_partition_id: str - app_key: str - - @classmethod - def populate(cls, ctx: dict) -> 'Context': - ctx_payload = ctx.pop('Payload') - app_key = ctx_payload['AppKey'] - data_partition_id = ctx_payload['data-partition-id'] - ctx_obj = cls(app_key=app_key, data_partition_id=data_partition_id) - return ctx_obj - - -class OSDURefResolver(jsonschema.RefResolver): - - def __init__(self, schema_service: str, *args, **kwargs): - super(OSDURefResolver, self).__init__(*args, **kwargs) - self.schema_service = schema_service - - def resolve_fragment(self, document: dict, fragment: str) -> dict: - """ - Extend base resolve_fragment method. If a nested schema has 'definitions' field and there - is a schema under this 'definitions', jsonschema attempts to use the id field of this - double-nested schema as URI to get this schema later. So it has sense to replace this id - with a correct one. - """ - document = super().resolve_fragment(document, fragment) - fragment_parts = fragment.split("/") # /definitions/<OsduID> -> [..., <OsduID>] - if len(fragment_parts) > 1: - osdu_id = fragment_parts[-1] - url = f"{self.schema_service}/{osdu_id}" - document["$id"] = url - return document - - -class SchemaValidator(object): - """Class to validate schema of Manifests.""" - - def __init__(self, schema_service: str, dagrun_conf: dict, context: Context): - self.schema_service = schema_service - self.data_object = copy.deepcopy(dagrun_conf) - self.context = context - self.resolver_handlers = {"osdu": self.get_schema_request, - "https": self.get_schema_request, - self.context.data_partition_id: self.get_schema_request} - self.create_request_headers() - - def create_request_headers(self): - self.request_headers = { - 'Content-type': 'application/json', - 'data-partition-id': self.context.data_partition_id, - 'AppKey': self.context.app_key, - } - - @refresh_token(AirflowTokenRefresher()) - def _get_schema_request(self, headers: dict, uri: str) -> requests.Response: - """ - Request to Schema service to retrieve schema. - """ - response = requests.get(uri, headers=headers, timeout=60) - return response - - def get_schema_request(self, uri: str) -> dict: - """ - Get Schema from Schema service. Change $id field to url of getting schema. - """ - if uri.startswith("osdu") or uri.startswith(self.context.data_partition_id): - uri = f"{self.schema_service}/{uri}" - response = self._get_schema_request(self.request_headers, uri).json() - response["$id"] = uri - return response - - def get_schema(self, kind: str) -> dict: - manifest_schema_uri = f"{self.schema_service}/{kind}" - try: - response = self.get_schema_request(manifest_schema_uri) - except Exception as e: - logger.error(f"Error on getting schema of kind '{kind}'") - raise e - return response - - def _validate_schema(self, manifest: dict, schema: dict = None): - """ - Validate schema. If argument schema is not defined, then use schema service to retrieve - corresponding schema. - """ - if not schema: - schema = self.get_schema(manifest["kind"]) - logger.info(f"Validating kind {manifest['kind']}") - resolver = OSDURefResolver(schema_service=self.schema_service, - base_uri=schema.get("$id", ""), referrer=schema, - handlers=self.resolver_handlers, cache_remote=True) - validator = jsonschema.Draft7Validator(schema=schema, resolver=resolver) - validator.validate(manifest) - - def validate_work_product(self, work_product: dict): - """ - Validate WP manifest. Raise error if manifest is not valid. - """ - for key, value in work_product.items(): - if key != "WorkProduct": - for component in value: - self._validate_schema(component) - else: - self._validate_schema(value) - - def validate_manifest(self): - """ - Validate manifest. Raise error if manifest is not valid. - """ - for manifest in self.data_object["manifest"]: - if isinstance(manifest, dict) and manifest.get("kind"): - self._validate_schema(manifest) - elif manifest.get("WorkProductComponents"): - self.validate_work_product(manifest) - else: - raise NotOSDUShemaFormatError(f"Not valid schema {manifest}") - - -class ManifestProcessor(object): - """Class to process WP, Master and Reference data""" - RECORD_TEMPLATE = { - "legal": {}, - "acl": {}, - "kind": "", - "id": "", - "data": { - } - } - - def __init__(self, storage_url, dagrun_conf, context): - self.storage_url = storage_url - self.data_object = copy.deepcopy(dagrun_conf) - self.context = context - - @staticmethod - def _get_kind_name(kind: str) -> str: - """ - osdu:osdu:Well:1.0.0 -> Well - """ - kind_name = kind.split(":")[2] - return kind_name - - def generate_id(self, manifest_fragment: dict) -> str: - """ - Generate id to use it in Storage. - """ - group_type = manifest_fragment.get("groupType", "doc") - kind = manifest_fragment.get("kind") - kind_name = self._get_kind_name(kind) - _id = f"{self.context.data_partition_id}:{group_type}_{kind_name}:{str(uuid.uuid4())}" - return _id - - @property - def request_headers(self) -> dict: - headers = { - 'Content-type': 'application/json', - 'data-partition-id': self.context.data_partition_id, - 'AppKey': self.context.app_key - } - return headers - - def populate_manifest_storage_record(self, manifest: dict) -> dict: - """ - Create a record from Master-manifest to store it in Storage service - """ - record = copy.deepcopy(self.RECORD_TEMPLATE) - record["id"] = self.generate_id(manifest) - record["kind"] = manifest.pop("kind") - record["legal"] = manifest.pop("legal") - record["acl"] = manifest.pop("acl") - record["data"] = manifest - return record - - @tenacity.retry(tenacity.wait_fixed(TIMEOUT), - tenacity.stop_after_attempt(RETRIES)) - @refresh_token(AirflowTokenRefresher()) - def save_record(self, headers: dict, request_data: List[dict]) -> requests.Response: - """ - Send request to record storage API. - """ - response = requests.put(self.storage_url, json.dumps(request_data), headers=headers) - if response.ok: - logger.info(",".join(map(str, response.json()["recordIds"]))) - else: - reason = response.text[:250] - logger.error(f"Request error.") - logger.error(f"Response status: {response.status_code}. " - f"Response content: {reason}.") - return response - - def process_work_product(self, manifest: dict) -> List[dict]: - """ - Process WP. - """ - wp = manifest["WorkProduct"] - records = [self.populate_manifest_storage_record(wp)] - return records - - def process_work_product_components(self, manifest: dict) -> List[dict]: - """ - Process list of WPS. - """ - records = [] - for wpc in manifest["WorkProductComponents"]: - record = self.populate_manifest_storage_record(wpc) - records.append(record) - return records - - def process_work_product_files(self, manifest: dict) -> List[dict]: - """ - Process list of files. - """ - records = [] - for file in manifest["Files"]: - record = self.populate_manifest_storage_record(file) - records.append(record) - return records - - def process_work_product_manifest(self, manifest: dict) -> List[dict]: - file_records = self.process_work_product_files(manifest) - wpc_records = self.process_work_product_components(manifest) - wp_records = self.process_work_product(manifest) - records = file_records + wpc_records + wp_records - return records - - def create_manifest_records(self) -> List[dict]: - manifest_records = [] - manifests = self.data_object["manifest"] - for manifest in manifests: - if "WorkProduct" in manifest: - wp_records = self.process_work_product_manifest(manifest) - manifest_records.extend(wp_records) - else: - record = self.populate_manifest_storage_record(manifest) - manifest_records.append(record) - return manifest_records - - def process_manifest(self) -> List[str]: - if "manifest" in self.data_object: - manifest_records = self.create_manifest_records() - else: - raise EmptyManifestError - response = self.save_record(self.request_headers, request_data=manifest_records) - record_ids = response.json()["recordIds"] - return record_ids - - class ProcessManifestOperatorR3(BaseOperator): ui_color = '#dad5ff' ui_fgcolor = '#000000' - def pre_execute(self, context: dict): + @apply_defaults + def __init__(self, *args, **kwargs) -> None: + super().__init__(*args, **kwargs) self.schema_service_url = Variable.get('schema_service_url') self.storage_url = Variable.get('storage_url') diff --git a/src/plugins/operators/search_record_id.py b/src/plugins/operators/search_record_id.py index ef8faedec6f731c98ce38120f737852a18fb2943..eb5b36679f0985b2188a8fb4025085bf0751e5e2 100644 --- a/src/plugins/operators/search_record_id.py +++ b/src/plugins/operators/search_record_id.py @@ -13,26 +13,13 @@ # See the License for the specific language governing permissions and # limitations under the License. - -import json import logging -import sys -from typing import Tuple -import tenacity from airflow.models import BaseOperator, Variable -from airflow.utils.decorators import apply_defaults -from hooks import search_http_hook, workflow_hook -from libs.exceptions import RecordsNotSearchableError -from libs.refresh_token import AirflowTokenRefresher, refresh_token +from libs.context import Context +from libs.search_record_ids import SearchId -# 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) +logger = logging.getLogger() class SearchRecordIdOperator(BaseOperator): @@ -47,82 +34,12 @@ class SearchRecordIdOperator(BaseOperator): RUNNING_STATUS = "running" FAILED_STATUS = "failed" - @apply_defaults - def __init__(self, *args, **kwargs) -> None: - super().__init__(*args, **kwargs) - self.workflow_hook = workflow_hook - self.search_hook = search_http_hook - # the will be set at the beginning of execute method - self.request_body = None - self.expected_total_count = None - - def get_headers(self, **kwargs) -> dict: - data_conf = kwargs['dag_run'].conf - # for /submitWithManifest authorization and partition-id are inside Payload field - if "Payload" in data_conf: - partition_id = data_conf["Payload"]["data-partition-id"] - else: - partition_id = data_conf["data-partition-id"] - headers = { - 'Content-type': 'application/json', - 'data-partition-id': partition_id, - 'Authorization': "", - } - return headers - - @staticmethod - def _create_search_query(record_ids) -> Tuple[str, int]: - expected_total_count = len(record_ids) - record_ids = " OR ".join(f"\"{id_}\"" for id_ in record_ids) - logger.info(f"Search query {record_ids}") - query = f"id:({record_ids})" - return query, expected_total_count - - def _create_request_body(self, **kwargs): - record_ids = kwargs["ti"].xcom_pull(key="record_ids", ) - if record_ids: - query, expected_total_count = self._create_search_query(record_ids) - else: - logger.error("There are no record ids") - sys.exit(2) - request_body = { - "kind": "*:*:*:*", - "query": query - } - return request_body, expected_total_count - - def _is_record_searchable(self, resp) -> bool: - """ - Check if search service returns expected totalCount of records. - """ - logger.info(resp.text) - data = resp.json() - return data.get("totalCount") == self.expected_total_count - - @refresh_token(AirflowTokenRefresher()) - def search_files(self, headers, **kwargs): - if self.request_body: - response = self.search_hook.run( - endpoint=Variable.get("search_query_ep"), - headers=headers, - data=json.dumps(self.request_body), - extra_options={"check_response": False} - ) - if not self._is_record_searchable(response): - logger.error("Expected amount (%s) of records not found." % - self.expected_total_count - ) - raise RecordsNotSearchableError - return response - else: - logger.error("There is an error in header or in request body") - sys.exit(2) - - def execute(self, context): + 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 """ - self.request_body, self.expected_total_count = self._create_request_body(**context) - headers = self.get_headers(**context) - self.search_files(headers, **context) + payload_context = Context.populate(context["dag_run"].conf) + record_ids = context["ti"].xcom_pull(key="record_ids", ) + ids_searcher = SearchId(Variable.get("search_url"), record_ids, payload_context, ) + ids_searcher.check_records_searchable() diff --git a/src/plugins/operators/update_status.py b/src/plugins/operators/update_status.py index 145911b0be693c587fe452fa1917aba700d83840..06eee9d6ef3af244af463aa0995ed5de1d77afdf 100644 --- a/src/plugins/operators/update_status.py +++ b/src/plugins/operators/update_status.py @@ -14,105 +14,36 @@ # limitations under the License. +import copy import enum -import json import logging -import sys -from functools import partial -import tenacity from airflow.models import BaseOperator, Variable -from airflow.utils.decorators import apply_defaults -from hooks.http_hooks import search_http_hook, workflow_hook +from libs.context import Context from libs.exceptions import PipelineFailedError -from libs.refresh_token import AirflowTokenRefresher, refresh_token +from libs.update_status import UpdateStatus -# 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) + +logger = logging.getLogger() class UpdateStatusOperator(BaseOperator): ui_color = '#10ECAA' ui_fgcolor = '#000000' - FINISHED_STATUS = "finished" - RUNNING_STATUS = "running" - FAILED_STATUS = "failed" - class prev_ti_state(enum.Enum): - NONE = enum.auto() - SUCCESS = enum.auto() - FAILED = enum.auto() + NONE = "running" + SUCCESS = "finished" + FAILED = "failed" - @apply_defaults - def __init__(self, *args, **kwargs) -> None: - super().__init__(*args, **kwargs) - self.workflow_hook = workflow_hook - self.search_hook = search_http_hook - - @staticmethod - def _file_searched(resp, expected_total_count): - """Check if search service returns totalCount. - The method is used as a callback + def get_previous_ti_statuses(self, context: dict): + """ + Get status of previous tasks' executions. + Return corresponding enum value. """ - data = resp.json() - return data.get("totalCount") == expected_total_count - - def get_headers(self, **kwargs): - data_conf = kwargs['dag_run'].conf - # for /submitWithManifest authorization and partition-id are inside Payload field - if "Payload" in data_conf: - partition_id = data_conf["Payload"]["data-partition-id"] - else: - partition_id = data_conf["data-partition-id"] - headers = { - 'Content-type': 'application/json', - 'data-partition-id': partition_id, - } - return headers - - def search_files(self, **kwargs): - def create_query(record_ids): - expected_total_count = len(record_ids) - record_ids = " OR ".join(f"\"{id_}\"" for id_ in record_ids) - logger.info(f"Search query {record_ids}") - query = f"id:({record_ids})" - return query, expected_total_count - - record_ids = kwargs["ti"].xcom_pull(key="record_ids", task_ids='create_records') - if record_ids: - query, expected_total_count = create_query(record_ids) - else: - logger.error("There are no record ids") - sys.exit(2) - headers = self.get_headers(**kwargs) - request_body = { - "kind": "*:*:*:*", - "query": query - } - retry_opts = { - "wait": tenacity.wait_exponential(multiplier=5), - "stop": tenacity.stop_after_attempt(5), - "retry": tenacity.retry_if_not_result( - partial(self._file_searched, expected_total_count=expected_total_count) - ) - } - self.search_hook.run_with_advanced_retry( - endpoint=Variable.get("search_query_ep"), - headers=headers, - data=json.dumps(request_body), - _retry_args=retry_opts - ) - - def previous_ti_statuses(self, context): dagrun = context['ti'].get_dagrun() - failed_ti, success_ti = dagrun.get_task_instances( - state='failed'), dagrun.get_task_instances(state='success') + 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 @@ -122,39 +53,31 @@ class UpdateStatusOperator(BaseOperator): 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): - prev_tis = self.previous_ti_statuses(context) - if prev_tis is self.prev_ti_state.NONE: - self.status = self.RUNNING_STATUS - elif prev_tis is self.prev_ti_state.FAILED: - self.status = self.FAILED_STATUS - elif prev_tis is self.prev_ti_state.SUCCESS: - self.status = self.FINISHED_STATUS + def pre_execute(self, context: dict): + self.status = self.get_previous_ti_statuses(context) - def execute(self, context): + 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 """ - headers = self.get_headers(**context) - self.update_status_request(headers, self.status, **context) - if self.status == self.FAILED_STATUS: + conf = copy.deepcopy(context["dag_run"].conf) + logger.info(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_url=Variable.get("update_status_url"), + workflow_id=workflow_id, + status=status, + context=payload_context + ) + status_updater.update_workflow_status() + if self.status is self.prev_ti_state.FAILED: raise PipelineFailedError("Dag failed") - @refresh_token(AirflowTokenRefresher()) - def update_status_request(self, headers, status, **kwargs): - data_conf = kwargs['dag_run'].conf - logger.info(f"Got dataconf {data_conf}") - workflow_id = data_conf["WorkflowID"] - request_body = { - "WorkflowID": workflow_id, - "Status": status - } - logger.info(f" Sending request '{status}'") - response = self.workflow_hook.run( - endpoint=Variable.get("update_status_ep"), - data=json.dumps(request_body), - headers=headers, - extra_options={"check_response": False} - ) - return response + diff --git a/tests/end-to-end-tests/mock-external-apis/app.py b/tests/end-to-end-tests/mock-external-apis/app.py index 23f54021fc968191b985ad4dbd95a9e351d2594e..7be8cd1d8820cabd999f85701847866b125d0a39 100644 --- a/tests/end-to-end-tests/mock-external-apis/app.py +++ b/tests/end-to-end-tests/mock-external-apis/app.py @@ -13,9 +13,20 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging +import sys from flask import Flask, json, request, url_for +# 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) + + OSDU_INGEST_SUCCES_FIFO = "/tmp/osdu_ingest_success" OSDU_INGEST_FAILED_FIFO = "/tmp/osdu_ingest_failed" @@ -27,7 +38,7 @@ def index(): @app.route('/st', methods=['POST', 'GET', "PUT"]) def storage(): - print(request.json) + logger.info(request.json) with open("/tmp/osdu_ingest_result", "w") as f: json.dump(request.get_json(), f) f.close() @@ -36,11 +47,11 @@ def storage(): @app.route('/wf/us', methods=['POST', 'GET', "PUT"]) def workflow(): - print(request) - print(request.headers) - print(request.json) - content = request.get_json(force=True) - print(content) + logger.info(request) + logger.info(request.headers) + logger.info(request.json) + content = request.json + logger.info(content) if content.get("Status") == "finished": None if content.get("Status") == "failed": diff --git a/tests/plugin-unit-tests/data/invalid/EmptyManifest.json b/tests/plugin-unit-tests/data/invalid/EmptyManifest.json new file mode 100644 index 0000000000000000000000000000000000000000..0a1b070a86bacba36ca6154e1e7dd9144c5d8572 --- /dev/null +++ b/tests/plugin-unit-tests/data/invalid/EmptyManifest.json @@ -0,0 +1,11 @@ +{ + "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", + "WorkflowID": "foo" +} diff --git a/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json b/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json new file mode 100644 index 0000000000000000000000000000000000000000..62b46151c77ce0fc7a8b851f824460b644c918fc --- /dev/null +++ b/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json @@ -0,0 +1,16 @@ +{ + "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": [ + { + + } + ], + "WorkflowID": "foo" +} 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 index 2aa7957e3c3d907d2df7cf840dd80b074adb55cc..b38869dbedaea6272fba4c8d87f9fc1d3d09889e 100644 --- a/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json +++ b/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json @@ -9,7 +9,7 @@ "$filename": "load_Wellbore.1.0.0_350112350400.json", "manifest": [ { - "id": "srn:opendes:master-data/Wellbore:350112350400", + "id": "opendes:master-data/Wellbore:350112350400", "kind": "opendes:osdu:Wellbore:0.3.0", "groupType": "master-data", "version": 1, diff --git a/tests/plugin-unit-tests/data/master/record_Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/record_Wellbore.0.3.0.json new file mode 100644 index 0000000000000000000000000000000000000000..a1e646613154df19c1012141c93a746ea97b5435 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/record_Wellbore.0.3.0.json @@ -0,0 +1,128 @@ +[ + { + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "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" + ] + }, + "kind": "opendes:osdu:Wellbore:0.3.0", + "id": "opendes:master-data/Wellbore:350112350400", + "data": { + "id": "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 + } + ] + } + } + } + } +] diff --git a/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json b/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..43f7e6e07f225840963f209ca2f99270d5edd450 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json @@ -0,0 +1,371 @@ +{ + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A hole in the ground extending from a point at the earth's surface to the maximum point of penetration.", + "additionalProperties": false, + "title": "Wellbore", + "type": "object", + "definitions": { + "opendes:osdu:AbstractMetaItem:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A meta data item, which allows the association of named properties or property values to a Unit/Measurement/CRS/Azimuth/Time context.", + "title": "Frame of Reference Meta Data Item", + "type": "object", + "properties": { + "persistableReference": { + "description": "The persistable reference string uniquely identifying the CRS or Unit.", + "title": "Persistable Reference", + "type": "string", + "example": "{\"scaleOffset\":{\"scale\":0.3048006096012192,\"offset\":0.0},\"symbol\":\"ftUS\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}" + }, + "kind": { + "description": "The kind of reference, unit, measurement, CRS or azimuth reference.", + "title": "Reference Kind", + "type": "string", + "enum": [ + "CRS", + "Unit", + "Measurement", + "AzimuthReference", + "DateTime" + ] + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides Unit/CRS context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"data.structureA.propertyB\" to define a unique context.", + "title": "Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "elevationFromMsl", + "totalDepthMdDriller", + "wellHeadProjected" + ] + }, + "name": { + "description": "The name of the CRS or the symbol/name of the unit.", + "title": "Name or Symbol", + "type": "string", + "example": [ + "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "ft" + ] + }, + "propertyValues": { + "description": "The list of property values, to which this meta data item provides Unit/CRS context to. Typically a unit symbol is a value to a data structure; this symbol is then registered in this propertyValues array and the persistableReference provides the absolute reference.", + "title": "Property Values", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "F", + "ftUS", + "deg" + ] + }, + "uncertainty": { + "description": "The uncertainty of the values measured given the unit or CRS unit.", + "title": "Uncertainty", + "type": "number" + } + }, + "required": [ + "kind", + "persistableReference" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractMetaItem.1.0.0.json" + }, + "opendes:osdu:AbstractLegalTags:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "Legal meta data like legal tags, relevant other countries, legal status.", + "additionalProperties": false, + "title": "Legal Meta Data", + "type": "object", + "properties": { + "legaltags": { + "description": "The list of legal tags.", + "title": "Legal Tags", + "type": "array", + "items": { + "type": "string" + } + }, + "otherRelevantDataCountries": { + "description": "The list of other relevant data countries.", + "title": "Other Relevant Data Countries", + "type": "array", + "items": { + "type": "string" + } + }, + "status": { + "description": "The legal status.", + "title": "Legal Status", + "type": "string" + } + }, + "required": [ + "legaltags", + "otherRelevantDataCountries" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalTags.1.0.0.json" + }, + "opendes:osdu:AbstractAccessControlList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "The access control tags associated with this entity.", + "additionalProperties": false, + "title": "Access Control List", + "type": "object", + "properties": { + "viewers": { + "description": "The list of viewers to which this data record is accessible/visible/discoverable.", + "title": "List of Viewers", + "type": "array", + "items": { + "type": "string" + } + }, + "owners": { + "description": "The list of owners of this data record.", + "title": "List of Owners", + "type": "array", + "items": { + "type": "string" + } + } + }, + "required": [ + "owners", + "viewers" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAccessControlList.1.0.0.json" + }, + "opendes:osdu:AbstractLegalParentList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A list of entity IDs in the data ecosystem, which act as legal parents to the current entity.", + "title": "Parent List", + "type": "object", + "properties": { + "parents": { + "description": "An array of none, one or many entity references in the data ecosystem, which identify the source of data in the legal sense. Example: the 'parents' will be queried when e.g. the subscription of source data services is terminated; access to the derivatives is also terminated.", + "title": "Parents", + "type": "array", + "items": { + "type": "string" + }, + "example": [] + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalParentList.1.0.0.json" + }, + "opendes:osdu:AbstractCoordinates:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A geographic position on the surface of the earth.", + "title": "AbstractCoordinates", + "type": "object", + "properties": { + "x": { + "description": "x is Easting or Longitude.", + "type": "number" + }, + "y": { + "description": "y is Northing or Latitude.", + "type": "number" + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractCoordinates.1.0.0.json" + } + }, + "properties": { + "groupType": { + "const": "master-data", + "description": "The OSDU GroupType assigned to this resource object.", + "title": "Group Type" + }, + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/opendes:osdu:AbstractLegalParentList:1.0.0" + }, + "resourceCurationStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceCurationStatus:[^:]+:[0-9]*$", + "description": "Describes the current Curation status.", + "title": "Resource Curation Status", + "type": "string" + }, + "licenseState": { + "pattern": "^srn:opendes:reference-data\/LicenseState:[^:]+:[0-9]*$", + "description": "Indicates what kind of ownership Company has over data.", + "title": "License State", + "type": "string" + }, + "data": { + "allOf": [ + { + "type": "object", + "properties": { + "VerticalMeasurements": { + "description": "List of all depths and elevations pertaining to the wellbore, like, plug back measured depth, total measured depth, KB elevation", + "type": "array" + }, + "PrimaryMaterialID": { + "pattern": "^srn:opendes:reference-data\/MaterialType:[^:]+:[0-9]*$", + "description": "The primary material injected/produced from the wellbore.", + "type": "string" + }, + "SequenceNumber": { + "description": "A number that indicates the order in which wellbores were drilled.", + "type": "integer" + }, + "TargetFormation": { + "pattern": "^srn:opendes:reference-data\/GeologicalFormation:[^:]+:[0-9]*$", + "description": "The Formation of interest for which the Wellbore is drilled to interact with. The Wellbore may terminate in a lower formation if the requirement is to drill through the entirety of the target formation, therefore this is not necessarily the Formation at TD.", + "type": "string" + }, + "KickOffWellbore": { + "pattern": "^srn:opendes:master-data\/Wellbore:[^:]+:[0-9]*$", + "description": "This is a pointer to the parent wellbore. The wellbore that starts from top has no parent.", + "type": "string" + }, + "DefaultVerticalMeasurementID": { + "description": "The default datum reference point, or zero depth point, used to determine other points vertically in a wellbore. References an entry in the Vertical Measurements array of this wellbore.", + "type": "string" + }, + + "WellID": { + "pattern": "^srn:opendes:master-data\/Well:[^:]+:[0-9]*$", + "type": "string" + }, + + "TrajectoryTypeID": { + "pattern": "^srn:opendes:reference-data\/WellboreTrajectoryType:[^:]+:[0-9]*$", + "description": "Describes the predominant shapes the wellbore path can follow if deviated from vertical. Sample Values: Horizontal, Vertical, Directional.", + "type": "string" + }, + "DefinitiveTrajectoryID": { + "pattern": "^srn:opendes:work-product-component\/WellboreTrajectory:[^:]+:[0-9]+$", + "description": "SRN of Wellbore Trajectory which is considered the authoritative or preferred version.", + "type": "string" + } + } + }, + { + "type": "object", + "properties": { + "ExtensionProperties": { + "type": "object", + "properties": {} + } + } + } + ] + }, + "kind": { + "pattern": "^[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[0-9]+.[0-9]+.[0-9]+$", + "description": "The schema identification for the OSDU resource object following the pattern <Namespace>:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.", + "title": "Entity Kind", + "type": "string", + "example": "namespace:osdu:Wellbore:2.7.112" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/opendes:osdu:AbstractAccessControlList:1.0.0" + }, + "source": { + "pattern": "^srn:opendes:master-data\/Organisation:[^:]+:[0-9]*$", + "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" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1831253916104085 + }, + "resourceVersionCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time when the current version of this resource entered the OSDU.", + "title": "Resource Version Creation DateTime", + "type": "string" + }, + "resourceHomeRegionID": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "description": "The name of the home [cloud environment] region for this OSDU resource object.", + "title": "Resource Home Region ID", + "type": "string" + }, + "resourceObjectCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time at which Version 1 of this OSDU resource object was originated.", + "title": "Resource Object Creation DateTime", + "type": "string" + }, + "resourceSecurityClassification": { + "pattern": "^srn:opendes:reference-data\/ResourceSecurityClassification:[^:]+:[0-9]*$", + "description": "Classifies the security level of the resource.", + "title": "Resource Security Classification", + "type": "string" + }, + "resourceHostRegionIDs": { + "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.", + "title": "Resource Host Region ID", + "type": "array", + "items": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "type": "string" + } + }, + "resourceLifecycleStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceLifecycleStatus:[^:]+:[0-9]*$", + "description": "Describes the current Resource Lifecycle status.", + "title": "Resource Lifecycle Status", + "type": "string" + }, + "legal": { + "description": "The entity's legal tags and compliance status.", + "title": "Legal Tags", + "$ref": "#/definitions/opendes:osdu:AbstractLegalTags:1.0.0" + }, + "id": { + "pattern": "^opendes:master-data/Wellbore:[^:]+$", + "description": "The SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "srn:opendes:master-data/Wellbore:2adac27b-5d84-5bcd-89f2-93ee709c06d9" + }, + "existenceKind": { + "pattern": "^srn:opendes:reference-data\/ExistenceKind:[^:]+:[0-9]*$", + "description": "Where does this data resource sit in the cradle-to-grave span of its existence?", + "title": "Existence Kind", + "type": "string" + }, + "persistableReferences": { + "description": "The meta data section linking the 'unitKey', 'crsKey' to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/opendes:osdu:AbstractMetaItem:1.0.0" + } + } + }, + "required": [ + "id", + "kind", + "groupType", + "version", + "legal", + "resourceObjectCreationDateTime", + "resourceVersionCreationDateTime" + ], + "$id": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json" +} diff --git a/tests/plugin-unit-tests/data/other/Fragment.json b/tests/plugin-unit-tests/data/other/Fragment.json new file mode 100644 index 0000000000000000000000000000000000000000..73c5261eeeead0fbe44ff00e685b2065f06a008f --- /dev/null +++ b/tests/plugin-unit-tests/data/other/Fragment.json @@ -0,0 +1,20 @@ +{ + "opendes:osdu:AbstractCoordinates:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A geographic position on the surface of the earth.", + "title": "AbstractCoordinates", + "type": "object", + "properties": { + "x": { + "description": "x is Easting or Longitude.", + "type": "number" + }, + "y": { + "description": "y is Northing or Latitude.", + "type": "number" + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractCoordinates.1.0.0.json" + } +} diff --git a/tests/plugin-unit-tests/data/other/SearchResponseInvalid.json b/tests/plugin-unit-tests/data/other/SearchResponseInvalid.json new file mode 100644 index 0000000000000000000000000000000000000000..de1abff81e6ebaae7d609d343f6b87fc28260a5d --- /dev/null +++ b/tests/plugin-unit-tests/data/other/SearchResponseInvalid.json @@ -0,0 +1,5 @@ +{ + "results": [ + ], + "aggregations": null +} diff --git a/tests/plugin-unit-tests/data/other/SearchResponseValid.json b/tests/plugin-unit-tests/data/other/SearchResponseValid.json new file mode 100644 index 0000000000000000000000000000000000000000..6e8fe92db01ccb71b541a6f204b45499414ec80f --- /dev/null +++ b/tests/plugin-unit-tests/data/other/SearchResponseValid.json @@ -0,0 +1,6 @@ +{ + "results": [ + ], + "aggregations": null, + "totalCount": 45 +} diff --git a/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json b/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json new file mode 100644 index 0000000000000000000000000000000000000000..5e8b95acaf851aee3222f9697cf4f7d3d3464115 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json @@ -0,0 +1,415 @@ +[ + { + "legal": { + "legaltags": [ + "legaltag" + ], + "otherRelevantDataCountries": [ + "NO", + "US" + ] + }, + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "kind": "opendes:osdu:File:1.0.0", + "id": "test_id", + "data": { + "groupType": "file", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", + "data": { + "SchemaFormatTypeID": "srn:opendes:reference-data/SchemaFormatType:SEG-Y Seismic Trace Data:", + "PreLoadFilePath": "C:\\Seismic\\ST0202R08_PS_PSDM_RAW_PP_TIME.MIG_RAW.POST_STACK.3D.JS-017534.segy", + "FileSource": "", + "FileSize": 277427976, + "EncodingFormatTypeID": "srn:opendes:reference-data/EncodingFormatType:segy:", + "Endian": "BIG", + "Checksum": "c9df7234d5d0a7a2c2676ee2e2aa48b6", + "VectorHeaderMapping": [ + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:INLINE:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 189 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CROSSLINE:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 193 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPX:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 181, + "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", + "ScalarIndicator": "OVERRIDE", + "ScalarOverride": 100.0 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPY:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 185, + "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", + "ScalarIndicator": "OVERRIDE", + "ScalarOverride": 100.0 + } + ] + }, + "AssociativeID": "f-1" + } + }, + { + "legal": { + "legaltags": [ + "legaltag" + ], + "otherRelevantDataCountries": [ + "NO", + "US" + ] + }, + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "kind": "opendes:osdu:SeismicTraceData:1.0.0", + "id": "test_id", + "data": { + "groupType": "work-product-component", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", + "meta": [ + { + "kind": "Unit", + "name": "ms", + "persistableReference": "{\"abcd\":{\"a\":0.0,\"b\":0.001,\"c\":1.0,\"d\":0.0},\"symbol\":\"ms\",\"baseMeasurement\":{\"ancestry\":\"T\",\"type\":\"UM\"},\"type\":\"UAD\"}", + "unitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:Energistics_UoM_ms:", + "propertyNames": [ + "StartTime", + "EndTime" + ] + }, + { + "kind": "Unit", + "name": "Amplitude", + "persistableReference": "{\"abcd\":{\"a\":0.0,\"b\":1.0,\"c\":1.0,\"d\":0.0},\"symbol\":\"Euc\",\"baseMeasurement\":{\"ancestry\":\"1\",\"type\":\"UM\"},\"type\":\"UAD\"}", + "unitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:Energistics_UoM_Euc:", + "propertyNames": [ + "RangeAmplitudeMax", + "RangeAmplitudeMin" + ] + } + ], + "data": { + "Name": "ST0202R08_PS_PSDM_RAW_PP_TIME.MIG_RAW", + "Description": "Seismic Trace Data", + "SpatialArea": { + "AsIngestedCoordinates": { + "type": "AnyCrsFeatureCollection", + "CoordinateReferenceSystemID": "srn:opendes:reference-data/CoordinateReferenceSystem:BoundCRS.SLB.23031.1613:", + "persistableReferenceCRS": "{\"lateBoundCRS\":{\"wkt\":\"PROJCS[\\\"ED_1950_UTM_Zone_31N\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Transverse_Mercator\\\"],PARAMETER[\\\"False_Easting\\\",500000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",3.0],PARAMETER[\\\"Scale_Factor\\\",0.9996],PARAMETER[\\\"Latitude_Of_Origin\\\",0.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",23031]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_UTM_Zone_31N\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"23031\"},\"type\":\"LBC\"},\"singleCT\":{\"wkt\":\"GEOGTRAN[\\\"ED_1950_To_WGS_1984_24\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],METHOD[\\\"Position_Vector\\\"],PARAMETER[\\\"X_Axis_Translation\\\",-90.365],PARAMETER[\\\"Y_Axis_Translation\\\",-101.13],PARAMETER[\\\"Z_Axis_Translation\\\",-123.384],PARAMETER[\\\"X_Axis_Rotation\\\",0.333],PARAMETER[\\\"Y_Axis_Rotation\\\",0.077],PARAMETER[\\\"Z_Axis_Rotation\\\",0.894],PARAMETER[\\\"Scale_Difference\\\",1.994],AUTHORITY[\\\"EPSG\\\",1613]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_To_WGS_1984_24\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"1613\"},\"type\":\"ST\"},\"ver\":\"PE_10_3_1\",\"name\":\"ED50 * EPSG-Nor S62 2001 / UTM zone 31N [23031,1613]\",\"authCode\":{\"auth\":\"SLB\",\"code\":\"23031024\"},\"type\":\"EBC\"}", + "features": [ + { + "type": "AnyCrsFeature", + "properties": null, + "geometry": { + "type": "AnyCrsPolygon", + "coordinates": [ + [ + [ + 438727.125, + 6475514.5 + ], + [ + 439888.34375, + 6480172.0 + ], + [ + 432562.59375, + 6481998.5 + ], + [ + 431401.375, + 6477341.0 + ], + [ + 438727.125, + 6475514.5 + ] + ] + ] + } + } + ] + }, + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "properties": null, + "geometry": { + "type": "Polygon", + "coordinates": [ + [ + [ + 1.9496878, + 58.4141503 + ], + [ + 1.9683366, + 58.4561357 + ], + [ + 1.8422866, + 58.4714655 + ], + [ + 1.8237804, + 58.4294624 + ], + [ + 1.9496878, + 58.4141503 + ] + ] + ] + } + } + ] + }, + "OperationsApplied": [ + "AsIngestedCoordinates converted to Wgs84Coordinates: Input CRS EPSG 23031 (ED50 / UTM zone 31N) to Target CRS EPSG 4326 (WGS84) using CT EPSG 1613 (ED50 to WGS 84 (24) - Norway - offshore south of 62\u00b0N - North Sea.)" + ], + "SpatialParameterTypeID": "srn:opendes:reference-data/SpatialParameterType:Outline:", + "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Polygon:" + }, + "LiveTraceOutline": { + "AsIngestedCoordinates": { + "type": "AnyCrsFeatureCollection", + "CoordinateReferenceSystemID": "srn:opendes:reference-data/CoordinateReferenceSystem:BoundCRS.SLB.23031.1613:", + "persistableReferenceCRS": "{\"lateBoundCRS\":{\"wkt\":\"PROJCS[\\\"ED_1950_UTM_Zone_31N\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Transverse_Mercator\\\"],PARAMETER[\\\"False_Easting\\\",500000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",3.0],PARAMETER[\\\"Scale_Factor\\\",0.9996],PARAMETER[\\\"Latitude_Of_Origin\\\",0.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",23031]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_UTM_Zone_31N\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"23031\"},\"type\":\"LBC\"},\"singleCT\":{\"wkt\":\"GEOGTRAN[\\\"ED_1950_To_WGS_1984_24\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],METHOD[\\\"Position_Vector\\\"],PARAMETER[\\\"X_Axis_Translation\\\",-90.365],PARAMETER[\\\"Y_Axis_Translation\\\",-101.13],PARAMETER[\\\"Z_Axis_Translation\\\",-123.384],PARAMETER[\\\"X_Axis_Rotation\\\",0.333],PARAMETER[\\\"Y_Axis_Rotation\\\",0.077],PARAMETER[\\\"Z_Axis_Rotation\\\",0.894],PARAMETER[\\\"Scale_Difference\\\",1.994],AUTHORITY[\\\"EPSG\\\",1613]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_To_WGS_1984_24\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"1613\"},\"type\":\"ST\"},\"ver\":\"PE_10_3_1\",\"name\":\"ED50 * EPSG-Nor S62 2001 / UTM zone 31N [23031,1613]\",\"authCode\":{\"auth\":\"SLB\",\"code\":\"23031024\"},\"type\":\"EBC\"}", + "features": [ + { + "type": "AnyCrsFeature", + "properties": null, + "geometry": { + "type": "AnyCrsPolygon", + "coordinates": [ + [ + [ + 438727.125, + 6475514.5 + ], + [ + 439888.34375, + 6480172.0 + ], + [ + 432562.59375, + 6481998.5 + ], + [ + 431401.375, + 6477341.0 + ], + [ + 438727.125, + 6475514.5 + ] + ] + ] + } + } + ] + }, + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "properties": null, + "geometry": { + "type": "Polygon", + "coordinates": [ + [ + [ + 1.9496878, + 58.4141503 + ], + [ + 1.9683366, + 58.4561357 + ], + [ + 1.8422866, + 58.4714655 + ], + [ + 1.8237804, + 58.4294624 + ], + [ + 1.9496878, + 58.4141503 + ] + ] + ] + } + } + ] + }, + "OperationsApplied": [ + "AsIngestedCoordinates converted to Wgs84Coordinates: Input CRS EPSG 23031 (ED50 / UTM zone 31N) to Target CRS EPSG 4326 (WGS84) using CT EPSG 1613 (ED50 to WGS 84 (24) - Norway - offshore south of 62\u00b0N - North Sea.)" + ], + "SpatialParameterTypeID": "srn:opendes:reference-data/SpatialParameterType:Outline:", + "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Polygon:" + }, + "PrincipalAcquisitionProjectID": "srn:opendes:master-data/SeismicAcquisitionProject:ST0202R08:", + "ProcessingProjectID": "srn:opendes:master-data/SeismicProcessingProject:ST0202R08:", + "SeismicTraceDataDimensionalityTypeID": "srn:opendes:reference-data/SeismicTraceDataDimensionalityType:3D:", + "SeismicDomainTypeID": "srn:opendes:reference-data/SeismicDomainType:Time:", + "SeismicMigrationTypeID": "srn:opendes:reference-data/SeismicMigrationType:Prestack Depth - Kirchhoff:", + "SeismicStackingTypeID": "srn:opendes:reference-data/SeismicStackingType:Full:", + "SeismicFilteringTypeID": "srn:opendes:reference-data/SeismicFilteringType:Tau-P:", + "Phase": "0", + "Polarity": "Normal", + "SampleInterval": 4.0, + "SampleCount": 1126, + "Difference": false, + "StartTime": 0.0, + "EndTime": 4500.0, + "TraceCount": 58479, + "TraceLength": 4500.0, + "TraceDomainUOM": "srn:opendes:reference-data/UnitOfMeasure:ms:", + "InlineMin": 9985, + "InlineMax": 10369, + "CrosslineMin": 1932, + "CrosslineMax": 2536, + "InlineIncrement": 2, + "CrosslineIncrement": 2, + "Precision": { + "WordFormat": "srn:opendes:reference-data/WordFormatType:IBM_FLOAT:", + "WordWidth": 4 + }, + "ProcessingParameters": [ + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:REFORMAT:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:NAVIGATION MERGE:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:TRACE EDITING:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:DESIGNATURE/ZEROPHASE:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:TIDAL STATICS:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:RESAMPLE:", + "ProcessingParameterValue": "4 MS" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:GEOPHONE MATCHING CORRECTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:GEOPHONE ORIENTATION CORRECTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:ROTATION TO RADIAL:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:NOISE BAND EDITING:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:RECEIVER SHEAR STATIC CORRECTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:TAUP DECONVOLUTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:3D KIRCHHOFF DEPTH MIGRATION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:VEL ANALYSIS AND RMO CORRECTION:", + "ProcessingParameterValue": "200M X 200M" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:CONVERSION TO PP TWT:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:PRE-RADON MILD DIP FILTER:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:PARABOLIC RADON TRANSFORM DEMULTIPLE:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:ANGLE MUTE:", + "ProcessingParameterValue": "3-37 DEGREES" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:STACK:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:OUTPUT SEGY:" + } + ], + "TextualFileHeader": [ + "C 1 CLIENT=STATOIL, PROCESSED BY=WESTERNGECO C 2 LINE ST0202D04-9985 C 3 SURVEY:ST0202 , AREA: VOLVE 15/9 C 4 DATAFORMAT: SEG-Y, DATE:02012008 C 5 DATATYPE: RAW PS PSDM STACK IN PP TIME C 6 DATA SHOT BY/VESSEL:GECO ANGLER, CABLE LENGTH:6000 M NO OF GROUPS: 240x4 C 7 NO OF CABLES 2, SOURCE:2, POP INTERVAL 25.0 M (FLIP-FLOP) C 8 RCV LINE SPACING: 400 M, SRC LINE SPACING: 100 M, RECORD LENGTH: 10.2 S C 9 GEODETIC DATUM: ED-50, SPHEROID: INTERNAT., PROJECTION: UTM C10 CENTR. MERID.: 03, UTM ZONE: 31 N, FALSE EASTING: 500000 C11 FIRST SAMPLE= 0, LAST SAMPLE= 4500, SAMPLE INTERVAL= 4 MS C12 DIST. BETWEEN INLINES=25.0M, XLINES=25.0M BIN SIZE (I x X):12.5M x 12.5M C13 INLINE DIRECTION (GRID): 284.0000000 DEGREES (CLOCKWISE FROM NORTH); C14 X-LINE DIRECTION (GRID): 014.0000000 DEGREES (CLOCKWISE FROM NORTH); C15 ORIGO(1.1) UTMX:431955.70E; UTMY:6348582.15N ; C16 DATA RANGE INLINES=9985-10369 (EVERY 2ND), X-LINES=1932-2536 (EVERY 2ND) C17 CORNER1:3D INLINE 9985, 3D XLINE 1932, UTM-X 438727.0, UTM-Y 6475514.4 C18 CORNER2:3D INLINE 9985, 3D XLINE 2536, UTM-X 431401.3, UTM-Y 6477341.0 C19 CORNER3:3D INLINE 10369, 3D XLINE 2536, UTM-X 432562.5, UTM-Y 6481998.4 C20 CORNER4:3D INLINE 10369, 3D XLINE 1932, UTM-X 439888.3, UTM-Y 6480171.9 C21 LIVE DATA POLYGON: (9985,1932);(9985,2536);(10369,2536);(10369,1932); C22 NAVIGATION SOURCE: P1/90 UKOOA BIN CENTER CELL GRID, C23 PROCESSING SEQUENCE: C24 1) REFORMAT 2) NAVIGATION MERGE 3) TRACE EDITING 4) DESIGNATURE/ZEROPHASE C25 5) TIDAL STATICS 6) RESAMPLE 4MS 7) GEOPHONE MATCHING CORRECTION C26 8) GEOPHONE ORIENTATION CORRECTION 9) ROTATION TO RADIAL C27 10) NOISE BAND EDITING 11) RECEIVER SHEAR STATIC CORRECTION C28 12) TAUP DECONVOLUTION 13) 3D KIRCHHOFF DEPTH MIGRATION C29 14) VEL ANALYSIS AND RMO CORRECTION 200M X 200M 15) CONVERSION TO PP TWT C30 16) PRE-RADON MILD DIP FILTER 17) PARABOLIC RADON TRANSFORM DEMULTIPLE C31 18) 3-37 DEGREE ANGLE MUTE 19) STACK 20) OUTPUT TO SEGY FORMAT C32 C33 C34 C35 HEADER WORD POSITIONS: C36 INLINE: 189-192 ; X-LINE: 193-196; C37 BINX (CDPX): 181-184, BINY (CDPY): 185-188, C38 MERID.: 3.0E, SPHEROID: INT.; ROTATION (AMS): 1245600000, C39 A POSITIVE SAMPLE CORRESPONDS TO A INCREASE IN ACOUSTIC IMPEDANCE. C40 END EBCDIC. " + ], + "RangeAmplitudeMax": 0.07441109418869019, + "RangeAmplitudeMin": -0.10446560382843018 + }, + "AssociativeID": "wpc-1", + "FileAssociativeIDs": [ + "f-1" + ] + } + }, + { + "legal": { + "legaltags": [ + "legaltag" + ], + "otherRelevantDataCountries": [ + "NO", + "US" + ] + }, + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "kind": "opendes:osdu:WorkProduct:1.0.0", + "id": "test_id", + "data": { + "groupType": "work-product", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", + "data": { + "Name": "ST0202R08_PS_PSDM_RAW_PP_TIME.MIG_RAW", + "Description": "Seismic Trace Data" + }, + "ComponentsAssociativeIDs": [ + "wpc-1" + ] + } + } +] diff --git a/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..ba5e926ebb17d760cbf7b8b76270b42150103a02 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json @@ -0,0 +1,402 @@ +{ + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "The generic file entity.", + "additionalProperties": false, + "title": "File", + "type": "object", + "definitions": { + "opendes:osdu:AbstractMetaItem:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A meta data item, which allows the association of named properties or property values to a Unit/Measurement/CRS/Azimuth/Time context.", + "title": "Frame of Reference Meta Data Item", + "type": "object", + "properties": { + "persistableReference": { + "description": "The persistable reference string uniquely identifying the CRS or Unit.", + "title": "Persistable Reference", + "type": "string", + "example": "{\"scaleOffset\":{\"scale\":0.3048006096012192,\"offset\":0.0},\"symbol\":\"ftUS\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}" + }, + "kind": { + "description": "The kind of reference, unit, measurement, CRS or azimuth reference.", + "title": "Reference Kind", + "type": "string", + "enum": [ + "CRS", + "Unit", + "Measurement", + "AzimuthReference", + "DateTime" + ] + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides Unit/CRS context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"data.structureA.propertyB\" to define a unique context.", + "title": "Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "elevationFromMsl", + "totalDepthMdDriller", + "wellHeadProjected" + ] + }, + "name": { + "description": "The name of the CRS or the symbol/name of the unit.", + "title": "Name or Symbol", + "type": "string", + "example": [ + "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "ft" + ] + }, + "propertyValues": { + "description": "The list of property values, to which this meta data item provides Unit/CRS context to. Typically a unit symbol is a value to a data structure; this symbol is then registered in this propertyValues array and the persistableReference provides the absolute reference.", + "title": "Property Values", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "F", + "ftUS", + "deg" + ] + }, + "uncertainty": { + "description": "The uncertainty of the values measured given the unit or CRS unit.", + "title": "Uncertainty", + "type": "number" + } + }, + "required": [ + "kind", + "persistableReference" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractMetaItem.1.0.0.json" + }, + "opendes:osdu:AbstractLegalTags:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "Legal meta data like legal tags, relevant other countries, legal status.", + "additionalProperties": false, + "title": "Legal Meta Data", + "type": "object", + "properties": { + "legaltags": { + "description": "The list of legal tags.", + "title": "Legal Tags", + "type": "array", + "items": { + "type": "string" + } + }, + "otherRelevantDataCountries": { + "description": "The list of other relevant data countries.", + "title": "Other Relevant Data Countries", + "type": "array", + "items": { + "type": "string" + } + }, + "status": { + "description": "The legal status.", + "title": "Legal Status", + "type": "string" + } + }, + "required": [ + "legaltags", + "otherRelevantDataCountries" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalTags.1.0.0.json" + }, + "opendes:osdu:AbstractAccessControlList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "The access control tags associated with this entity.", + "additionalProperties": false, + "title": "Access Control List", + "type": "object", + "properties": { + "viewers": { + "description": "The list of viewers to which this data record is accessible/visible/discoverable.", + "title": "List of Viewers", + "type": "array", + "items": { + "type": "string" + } + }, + "owners": { + "description": "The list of owners of this data record.", + "title": "List of Owners", + "type": "array", + "items": { + "type": "string" + } + } + }, + "required": [ + "owners", + "viewers" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAccessControlList.1.0.0.json" + }, + "opendes:osdu:AbstractLegalParentList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A list of entity IDs in the data ecosystem, which act as legal parents to the current entity.", + "title": "Parent List", + "type": "object", + "properties": { + "parents": { + "description": "An array of none, one or many entity references in the data ecosystem, which identify the source of data in the legal sense. Example: the 'parents' will be queried when e.g. the subscription of source data services is terminated; access to the derivatives is also terminated.", + "title": "Parents", + "type": "array", + "items": { + "type": "string" + }, + "example": [] + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalParentList.1.0.0.json" + } + }, + "properties": { + "groupType": { + "const": "file", + "description": "The OSDU GroupType assigned to this resource object.", + "title": "Group Type" + }, + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/opendes:osdu:AbstractLegalParentList:1.0.0" + }, + "resourceCurationStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceCurationStatus:[^:]+:[0-9]*$", + "description": "Describes the current Curation status.", + "title": "Resource Curation Status", + "type": "string" + }, + "licenseState": { + "pattern": "^srn:opendes:reference-data\/LicenseState:[^:]+:[0-9]*$", + "description": "Indicates what kind of ownership Company has over data.", + "title": "License State", + "type": "string" + }, + "data": { + "allOf": [ + { + "type": "object", + "properties": { + "Endian": { + "description": "Endianness of binary value. Enumeration: \"BIG\", \"LITTLE\". If absent, applications will need to interpret from context indicators.", + "type": "string", + "enum": [ + "BIG", + "LITTLE" + ] + }, + "LossyCompressionIndicator": { + "description": "Boolean that warns that an imperfect compression algorithm has been applied to the bulk binary data. Details of the compression method need to be discovered from the format properties and file access methods.", + "type": "boolean" + }, + "VectorHeaderMapping": { + "description": "Array of objects which define the meaning and format of a tabular structure used in a binary file as a header. The initial use case is the trace headers of a SEG-Y file. Note that some of this information may be repeated in the SEG-Y EBCDIC header.", + "type": "array", + "items": { + "type": "object", + "properties": { + "KeyName": { + "pattern": "^srn:opendes:reference-data\/HeaderKeyName:[^:]+:[0-9]*$", + "description": "SRN of a reference value for a name of a property header such as INLINE, CDPX.", + "type": "string" + }, + "UoM": { + "pattern": "^srn:opendes:reference-data\/UnitOfMeasure:[^:]+:[0-9]*$", + "description": "SRN to units of measure reference if header standard is not followed.", + "type": "string" + }, + "Position": { + "description": "Beginning byte position of header value, 1 indexed.", + "type": "integer" + }, + "ScalarOverride": { + "description": "Scalar value (as defined by standard) when a value present in the header needs to be overwritten for this value.", + "type": "number" + }, + "WordWidth": { + "description": "Size of the word in bytes.", + "type": "integer" + }, + "WordFormat": { + "pattern": "^srn:opendes:reference-data\/WordFormatType:[^:]+:[0-9]*$", + "description": "SRN of a reference value for binary data types, such as INT, UINT, FLOAT, IBM_FLOAT, ASCII, EBCDIC.", + "type": "string" + }, + "ScalarIndicator": { + "description": "Enumerated string indicating whether to use the normal scalar field for scaling this field (STANDARD), no scaling (NOSCALE), or override scalar (OVERRIDE). Default is current STANDARD (such as SEG-Y rev2).", + "type": "string", + "enum": [ + "STANDARD", + "NOSCALE", + "OVERRIDE" + ] + } + } + } + }, + "Checksum": { + "pattern": "^[0-9a-fA-F]{32}$", + "description": "MD5 checksum of file bytes - a 32 byte hexadecimal number", + "type": "string" + }, + "PreLoadFilePath": { + "description": "File system path to the data file as it existed before loading to the data platform", + "type": "string" + }, + "FileSource": { + "description": "URL or file path for the data in the file", + "type": "string" + }, + "CompressionLevel": { + "description": "Number indicating degree of fidelity present in bulk data resulting from compression. Meaning of number depends on algorithm.", + "type": "number" + }, + "EncodingFormatTypeID": { + "pattern": "^srn:opendes:reference-data\/EncodingFormatType:[^:]+:[0-9]*$", + "description": "Encoding Format Type ID", + "type": "string" + }, + "CompressionMethodTypeID": { + "pattern": "^srn:opendes:reference-data\/CompressionMethodType:[^:]+:[0-9]*$", + "description": "Name of a compression algorithm applied to the data as stored.", + "type": "string" + }, + "SchemaFormatTypeID": { + "pattern": "^srn:opendes:reference-data\/SchemaFormatType:[^:]+:[0-9]*$", + "description": "Schema Format Type ID", + "type": "string" + }, + "FileSize": { + "description": "Length of file in bytes", + "type": "integer" + } + } + }, + { + "type": "object", + "properties": { + "ExtensionProperties": { + "type": "object", + "properties": {} + } + } + } + ] + }, + "kind": { + "pattern": "^[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[0-9]+.[0-9]+.[0-9]+$", + "description": "The schema identification for the OSDU resource object following the pattern <Namespace>:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.", + "title": "Entity Kind", + "type": "string", + "example": "namespace:osdu:File:2.7.112" + }, + "AssociativeID": { + "type": "string" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/opendes:osdu:AbstractAccessControlList:1.0.0" + }, + "source": { + "pattern": "^srn:opendes:master-data\/Organisation:[^:]+:[0-9]*$", + "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" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1831253916104085 + }, + "resourceVersionCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time when the current version of this resource entered the OSDU.", + "title": "Resource Version Creation DateTime", + "type": "string" + }, + "resourceHomeRegionID": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "description": "The name of the home [cloud environment] region for this OSDU resource object.", + "title": "Resource Home Region ID", + "type": "string" + }, + "resourceObjectCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time at which Version 1 of this OSDU resource object was originated.", + "title": "Resource Object Creation DateTime", + "type": "string" + }, + "resourceSecurityClassification": { + "pattern": "^srn:opendes:reference-data\/ResourceSecurityClassification:[^:]+:[0-9]*$", + "description": "Classifies the security level of the resource.", + "title": "Resource Security Classification", + "type": "string" + }, + "resourceHostRegionIDs": { + "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.", + "title": "Resource Host Region ID", + "type": "array", + "items": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "type": "string" + } + }, + "resourceLifecycleStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceLifecycleStatus:[^:]+:[0-9]*$", + "description": "Describes the current Resource Lifecycle status.", + "title": "Resource Lifecycle Status", + "type": "string" + }, + "legal": { + "description": "The entity's legal tags and compliance status.", + "title": "Legal Tags", + "$ref": "#/definitions/opendes:osdu:AbstractLegalTags:1.0.0" + }, + "id": { + "pattern": "^srn:opendes:file\/File:[^:]+$", + "description": "The SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "srn:opendes:file/File:6039b91f-04a5-5c02-b4ed-413f565e561c" + }, + "existenceKind": { + "pattern": "^srn:opendes:reference-data\/ExistenceKind:[^:]+:[0-9]*$", + "description": "Where does this data resource sit in the cradle-to-grave span of its existence?", + "title": "Existence Kind", + "type": "string" + }, + "persistableReferences": { + "description": "The meta data section linking the 'unitKey', 'crsKey' to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/opendes:osdu:AbstractMetaItem:1.0.0" + } + } + }, + "required": [ + "kind", + "groupType", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/file/File.1.0.0.json" +} diff --git a/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..7a7e2bd79ef6d5b210cf75b4779726057a87ca43 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json @@ -0,0 +1,598 @@ +{ + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A single logical dataset containing seismic samples.", + "additionalProperties": false, + "title": "SeismicTraceData", + "type": "object", + "definitions": { + "opendes:osdu:AbstractMetaItem:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A meta data item, which allows the association of named properties or property values to a Unit/Measurement/CRS/Azimuth/Time context.", + "title": "Frame of Reference Meta Data Item", + "type": "object", + "properties": { + "persistableReference": { + "description": "The persistable reference string uniquely identifying the CRS or Unit.", + "title": "Persistable Reference", + "type": "string", + "example": "{\"scaleOffset\":{\"scale\":0.3048006096012192,\"offset\":0.0},\"symbol\":\"ftUS\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}" + }, + "kind": { + "description": "The kind of reference, unit, measurement, CRS or azimuth reference.", + "title": "Reference Kind", + "type": "string", + "enum": [ + "CRS", + "Unit", + "Measurement", + "AzimuthReference", + "DateTime" + ] + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides Unit/CRS context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"data.structureA.propertyB\" to define a unique context.", + "title": "Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "elevationFromMsl", + "totalDepthMdDriller", + "wellHeadProjected" + ] + }, + "name": { + "description": "The name of the CRS or the symbol/name of the unit.", + "title": "Name or Symbol", + "type": "string", + "example": [ + "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "ft" + ] + }, + "propertyValues": { + "description": "The list of property values, to which this meta data item provides Unit/CRS context to. Typically a unit symbol is a value to a data structure; this symbol is then registered in this propertyValues array and the persistableReference provides the absolute reference.", + "title": "Property Values", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "F", + "ftUS", + "deg" + ] + }, + "uncertainty": { + "description": "The uncertainty of the values measured given the unit or CRS unit.", + "title": "Uncertainty", + "type": "number" + } + }, + "required": [ + "kind", + "persistableReference" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractMetaItem.1.0.0.json" + }, + "opendes:osdu:AbstractLegalTags:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "Legal meta data like legal tags, relevant other countries, legal status.", + "additionalProperties": false, + "title": "Legal Meta Data", + "type": "object", + "properties": { + "legaltags": { + "description": "The list of legal tags.", + "title": "Legal Tags", + "type": "array", + "items": { + "type": "string" + } + }, + "otherRelevantDataCountries": { + "description": "The list of other relevant data countries.", + "title": "Other Relevant Data Countries", + "type": "array", + "items": { + "type": "string" + } + }, + "status": { + "description": "The legal status.", + "title": "Legal Status", + "type": "string" + } + }, + "required": [ + "legaltags", + "otherRelevantDataCountries" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalTags.1.0.0.json" + }, + "opendes:osdu:AbstractAccessControlList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "The access control tags associated with this entity.", + "additionalProperties": false, + "title": "Access Control List", + "type": "object", + "properties": { + "viewers": { + "description": "The list of viewers to which this data record is accessible/visible/discoverable.", + "title": "List of Viewers", + "type": "array", + "items": { + "type": "string" + } + }, + "owners": { + "description": "The list of owners of this data record.", + "title": "List of Owners", + "type": "array", + "items": { + "type": "string" + } + } + }, + "required": [ + "owners", + "viewers" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAccessControlList.1.0.0.json" + }, + "opendes:osdu:AbstractLegalParentList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A list of entity IDs in the data ecosystem, which act as legal parents to the current entity.", + "title": "Parent List", + "type": "object", + "properties": { + "parents": { + "description": "An array of none, one or many entity references in the data ecosystem, which identify the source of data in the legal sense. Example: the 'parents' will be queried when e.g. the subscription of source data services is terminated; access to the derivatives is also terminated.", + "title": "Parents", + "type": "array", + "items": { + "type": "string" + }, + "example": [] + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalParentList.1.0.0.json" + } + }, + "properties": { + "groupType": { + "const": "work-product-component", + "description": "The OSDU GroupType assigned to this resource object.", + "title": "Group Type" + }, + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/opendes:osdu:AbstractLegalParentList:1.0.0" + }, + "resourceCurationStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceCurationStatus:[^:]+:[0-9]*$", + "description": "Describes the current Curation status.", + "title": "Resource Curation Status", + "type": "string" + }, + "licenseState": { + "pattern": "^srn:opendes:reference-data\/LicenseState:[^:]+:[0-9]*$", + "description": "Indicates what kind of ownership Company has over data.", + "title": "License State", + "type": "string" + }, + "data": { + "allOf": [ + { + "type": "object", + "properties": { + "StackAzimuthRangeMax": { + "description": "The stacking azimuth range maximum used during processing of this trace data set.", + "type": "number" + }, + "CrosslineMin": { + "description": "Smallest Xline/Cross line/Bin Value", + "type": "number" + }, + "CrosslineIncrement": { + "description": "Sampling interval of crosslines in this dataset (difference in labels between neighboring crosslines).", + "type": "number" + }, + "FirstCMP": { + "description": "First Common Mid Point", + "type": "number" + }, + "Difference": { + "description": "Indicates if the volume is a product of the difference between 4D surveys", + "type": "boolean" + }, + "FirstShotPoint": { + "description": "The shotpoint that came before all others", + "type": "number" + }, + "VerticalDatumUOM": { + "pattern": "^srn:opendes:reference-data\/UnitOfMeasure:[^:]+:[0-9]*$", + "description": "Unit of measure for vertical datum", + "type": "string" + }, + "SeismicFilteringTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicFilteringType:[^:]+:[0-9]*$", + "description": "ID of the Seismic Filtering Type", + "type": "string" + }, + "InlineMin": { + "description": "Smallest Inline/Line/Track value", + "type": "number" + }, + "TextualFileHeader": { + "description": "Character metadata from headers inside file, such as the EBCDIC header of SEGY. This is an array to capture each stanza separately.", + "type": "array", + "items": { + "type": "string" + } + }, + "VerticalMeasurementTypeID": { + "pattern": "^srn:opendes:reference-data\/VerticalMeasurementType:[^:]+:[0-9]*$", + "description": "Identifies a vertical reference datum type. E.g. mean sea level, ground level, mudline.", + "type": "string" + }, + "SampleCount": { + "description": "Number of samples in the vertical direction.", + "type": "integer" + }, + "SeismicStackingTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicStackingType:[^:]+:[0-9]*$", + "description": "ID of the Seismic Stacking Type", + "type": "string" + }, + "ProcessingProjectID": { + "pattern": "^srn:opendes:master-data\/SeismicProcessingProject:[^:]+:[0-9]*$", + "description": "The processing project from which this trace dataset was produced. Absent for field data.", + "type": "string" + }, + "SeismicDomainTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicDomainType:[^:]+:[0-9]*$", + "description": "ID of the nature of the vertical axis in the trace data set, usually Depth or Time.", + "type": "string" + }, + "ReplacementVelocityUOM": { + "pattern": "^srn:opendes:reference-data\/UnitOfMeasure:[^:]+:[0-9]*$", + "description": "Replacement Velocity Unit of Measure", + "type": "string" + }, + "ProcessingParameters": { + "description": "Processing Parameters to simply capture process history until full provenance model can be implemented.", + "type": "array", + "items": { + "type": "object", + "properties": { + "ProcessingParameterValue": { + "description": "The quantity for the processing parameter. May include units, ordering, and other descriptions.", + "type": "string" + }, + "ProcessingParameterTypeID": { + "pattern": "^srn:opendes:reference-data\/ProcessingParameterType:[^:]+:[0-9]*$", + "description": "Processing Parameter Type", + "type": "string" + } + } + } + }, + "HorizontalCRSID": { + "pattern": "^srn:opendes:reference-data\/HorizontalCRS:[^:]+:[0-9]*$", + "description": "Coordinate reference system of positions in trace header, which matches what is described in BinGrid but is repeated here for convenience and in case bin grid is not defined. In case of conflict with Bin Grid, this value applies to the coordinates as written into CMPX, CMPY headers. Nevertheless, Bin Grid should be used for mapping purposes.", + "type": "string" + }, + "SeismicAttributeTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicAttributeType:[^:]+:[0-9]*$", + "description": "ID of the Seismic Trace Data Type", + "type": "string" + }, + "InlineMax": { + "description": "Largest Inline/Line/Track value", + "type": "number" + }, + "BinGridID": { + "pattern": "^srn:opendes:work-product-component\/SeismicBinGrid:[^:]+:[0-9]+$", + "description": "Reference to the WPC which describes the node positions of the processed bin centers. These are indexed from the trace file using inline and crossline. ", + "type": "string" + }, + "SeismicProcessingStageTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicProcessingStageType:[^:]+:[0-9]*$", + "description": "It specifies if the seismic is as acquired, or has been manipulated by a process or workflow before arriving at its current state.", + "type": "string" + }, + "StartTime": { + "description": "The sample axis value in the vertical dimension at which Time formatted data starts. Use SeismicDomainType to know which of time or depth pairs is the actual range vs. what is estimated.", + "type": "number" + }, + "StartDepth": { + "description": "The sample axis value in the vertical dimension at which Depth formatted data starts. Use SeismicDomainType to know which of time or depth pairs is the actual range vs. what is estimated.", + "type": "number" + }, + "LiveTraceOutline": { + "description": "Polygon showing the coverage of live traces in the trace dataset", + "type": "object" + }, + "CrosslineMax": { + "description": "Largest Xline/Cross line/Bin Value", + "type": "number" + }, + "StackAzimuthRangeMin": { + "description": "The stacking azimuth range minimum used during processing of this trace data set.", + "type": "number" + }, + "Precision": { + "description": "Sample data format in terms of sample value precision 8bit Integer, 16bit Floating Point etc.", + "type": "object", + "properties": { + "WordWidth": { + "description": "Size of the word in bytes.", + "type": "integer" + }, + "WordFormat": { + "pattern": "^srn:opendes:reference-data\/WordFormatType:[^:]+:[0-9]*$", + "description": "SRN of a reference value for binary data types, such as INT, UINT, FLOAT, IBM_FLOAT, ASCII, EBCDIC.", + "type": "string" + } + } + }, + "SeismicWaveTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicWaveType:[^:]+:[0-9]*$", + "description": "The observed wave mode type in this trace data set (P, Sv, etc).", + "type": "string" + }, + "SampleInterval": { + "description": "Vertical sampling interval of data", + "type": "number" + }, + "TraceDomainUOM": { + "pattern": "^srn:opendes:reference-data\/UnitOfMeasure:[^:]+:[0-9]*$", + "description": "UOM for vertical trace domain values", + "type": "string" + }, + "PrincipalAcquisitionProjectID": { + "pattern": "^srn:opendes:master-data\/SeismicAcquisitionProject:[^:]+:[0-9]*$", + "description": "For most datasets, the acquisition project that generated the underlying field data. For merges, probably absent (see processing project for set of acquisition projects used in processing this dataset).", + "type": "string" + }, + "TraceCount": { + "description": "How many traces are in the volume", + "type": "integer" + }, + "Phase": { + "description": "Amount of phase rotation applied to data", + "type": "string" + }, + "CoveragePercent": { + "description": "Actual nominal fold of the trace data set as processed, expressed as the mode in percentage points (60 fold = 6000%).", + "type": "number" + }, + "EndTime": { + "description": "The sample axis value in the vertical dimension at which Time formatted data starts. Use SeismicDomainType to know which of time or depth pairs is the actual range vs. what is estimated.", + "type": "number" + }, + "StackOffsetRangeMax": { + "description": "The stacking offset range maximum used during processing of this trace data set.", + "type": "number" + }, + "SeismicTraceDataDimensionalityTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicTraceDataDimensionalityType:[^:]+:[0-9]*$", + "description": "The dimensionality of trace data sets (not as acquired but as in the dataset), such as 2D, 3D, 4D.", + "type": "string" + }, + "StackAngleRangeMax": { + "description": "The stacking angle range maximum used during processing of this trace data set.", + "type": "number" + }, + "SeismicLineGeometryID": { + "pattern": "^srn:opendes:work-product-component\/SeismicLineGeometry:[^:]+:[0-9]+$", + "description": "Reference to Seismic Line Geometry for 2D seismic processing which describes the node positions of the processed bin centers. These are indexed from the trace file using CMP (not trace number).", + "type": "string" + }, + "TraceRelationFileID": { + "pattern": "^srn:opendes:file\/File:[^:]+:[0-9]*$", + "description": "The SRN of a file within the WPC that shows the relationship between trace index and CMP number if the trace data file CMP header is unreliable (for 2D).", + "type": "string" + }, + "Polarity": { + "description": "Reflection polarity of embedded wavelet. Normal, Reverse, Plus 90, Minus 90 according to SEG standard.", + "type": "string" + }, + "LastCMP": { + "description": "Last Common Mid Point", + "type": "number" + }, + "StackAngleRangeMin": { + "description": "The stacking angle range minimum used during processing of this trace data set.", + "type": "number" + }, + "Preferred2DInterpretationSurveyID": { + "pattern": "^srn:opendes:master-data\/Seismic2DInterpretationSurvey:[^:]+:[0-9]*$", + "description": "For a 2D line section, SRN of the 2D Seismic Interpretation Survey which can be considered the master for the area and of which this trace dataset is a privileged member. It defines the set of trace datasets of a particular cohesive set of 2D processing geometries in a survey area that form the best set for interpretation (not an acquisition survey).", + "type": "string" + }, + "Preferred3DInterpretationSurveyID": { + "pattern": "^srn:opendes:master-data\/Seismic3DInterpretationSurvey:[^:]+:[0-9]*$", + "description": "For a 3D volume (including linear subsets), SRN of the 3D Seismic Interpretation Survey which can be considered the master for the area and of which this trace dataset is a privileged member. It defines the set of trace datasets of a particular bin grid that form the best set for interpretation (not an acquisition survey).", + "type": "string" + }, + "TraceLength": { + "description": "Maximum trace length calculated using depth or time start and end points as appropriate according to SeismicDomainType.", + "type": "number" + }, + "LastShotPoint": { + "description": "The last shotpoint represented by the data", + "type": "number" + }, + "ShiftApplied": { + "description": "Indicates how much the data has been shifted from the Vertical Datum (seismic reference datum) in the domain and units of SeismicDomainType and in the sense that a positive number causes a sample to move downward in physical space (lower elevation).", + "type": "string" + }, + "RangeAmplitudeMin": { + "description": "The actual minimum amplitude value found in the dataset.", + "type": "number" + }, + "VerticalDatumOffset": { + "description": "Datum value, the elevation of zero time/depth on the vertical axis in the domain of SeismicDomainType relative to the vertical reference datum used (usually MSL). Positive is upward from zero elevation to seismic datum).", + "type": "number" + }, + "InlineIncrement": { + "description": "Sampling interval of inlines in this dataset (difference in labels between neighboring inlines).", + "type": "number" + }, + "Seismic2DName": { + "description": "2D line name or survey name for the 2D group", + "type": "string" + }, + "StackOffsetRangeMin": { + "description": "The stacking offset range minimum used during processing of this trace data set.", + "type": "number" + }, + "SeismicMigrationTypeID": { + "pattern": "^srn:opendes:reference-data\/SeismicMigrationType:[^:]+:[0-9]*$", + "description": "ID of the Seismic Migration Data Type", + "type": "string" + }, + "EndDepth": { + "description": "The sample axis value in the vertical dimension at which Depth formatted data ends. Use SeismicDomainType to know which of time or depth pairs is the actual range vs. what is estimated.", + "type": "number" + }, + "ReplacementVelocity": { + "description": "Value used to produce vertical static shifts in data", + "type": "number" + }, + "RangeAmplitudeMax": { + "description": "The actual maximum amplitude value found in the dataset.", + "type": "number" + } + } + }, + { + "type": "object", + "properties": { + "ExtensionProperties": { + "type": "object", + "properties": {} + } + } + } + ] + }, + "kind": { + "pattern": "^[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[0-9]+.[0-9]+.[0-9]+$", + "description": "The schema identification for the OSDU resource object following the pattern <Namespace>:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.", + "title": "Entity Kind", + "type": "string", + "example": "namespace:osdu:SeismicTraceData:2.7.112" + }, + "FileAssociativeIDs": { + "description": "ID of Files", + "type": "array", + "items": { + "type": "string" + } + }, + "AssociativeID": { + "type": "string" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/opendes:osdu:AbstractAccessControlList:1.0.0" + }, + "source": { + "pattern": "^srn:opendes:master-data\/Organisation:[^:]+:[0-9]*$", + "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" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1831253916104085 + }, + "resourceVersionCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time when the current version of this resource entered the OSDU.", + "title": "Resource Version Creation DateTime", + "type": "string" + }, + "resourceHomeRegionID": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "description": "The name of the home [cloud environment] region for this OSDU resource object.", + "title": "Resource Home Region ID", + "type": "string" + }, + "resourceObjectCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time at which Version 1 of this OSDU resource object was originated.", + "title": "Resource Object Creation DateTime", + "type": "string" + }, + "resourceSecurityClassification": { + "pattern": "^srn:opendes:reference-data\/ResourceSecurityClassification:[^:]+:[0-9]*$", + "description": "Classifies the security level of the resource.", + "title": "Resource Security Classification", + "type": "string" + }, + "resourceHostRegionIDs": { + "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.", + "title": "Resource Host Region ID", + "type": "array", + "items": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "type": "string" + } + }, + "meta": { + "type": "array" + }, + "resourceLifecycleStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceLifecycleStatus:[^:]+:[0-9]*$", + "description": "Describes the current Resource Lifecycle status.", + "title": "Resource Lifecycle Status", + "type": "string" + }, + "legal": { + "description": "The entity's legal tags and compliance status.", + "title": "Legal Tags", + "$ref": "#/definitions/opendes:osdu:AbstractLegalTags:1.0.0" + }, + "id": { + "description": "The SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "srn:opendes:work-product-component/SeismicTraceData:1f855537-dea8-5b2a-a6d6-63fc9a0bbac1" + }, + "existenceKind": { + "pattern": "^srn:opendes:reference-data\/ExistenceKind:[^:]+:[0-9]*$", + "description": "Where does this data resource sit in the cradle-to-grave span of its existence?", + "title": "Existence Kind", + "type": "string" + }, + "persistableReferences": { + "description": "The meta data section linking the 'unitKey', 'crsKey' to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/opendes:osdu:AbstractMetaItem:1.0.0" + } + } + }, + "required": [ + "kind", + "groupType", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/work-product-component/SeismicTraceData.1.0.0.json" +} diff --git a/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..f2c69e41474742506861416dc4ea074fff171b6e --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json @@ -0,0 +1,403 @@ +{ + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A collection of work product components such as might be produced by a business activity and which is delivered to the data platform for loading.", + "additionalProperties": false, + "title": "WorkProduct", + "type": "object", + "definitions": { + "opendes:osdu:AbstractMetaItem:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A meta data item, which allows the association of named properties or property values to a Unit/Measurement/CRS/Azimuth/Time context.", + "title": "Frame of Reference Meta Data Item", + "type": "object", + "properties": { + "persistableReference": { + "description": "The persistable reference string uniquely identifying the CRS or Unit.", + "title": "Persistable Reference", + "type": "string", + "example": "{\"scaleOffset\":{\"scale\":0.3048006096012192,\"offset\":0.0},\"symbol\":\"ftUS\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}" + }, + "kind": { + "description": "The kind of reference, unit, measurement, CRS or azimuth reference.", + "title": "Reference Kind", + "type": "string", + "enum": [ + "CRS", + "Unit", + "Measurement", + "AzimuthReference", + "DateTime" + ] + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides Unit/CRS context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"data.structureA.propertyB\" to define a unique context.", + "title": "Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "elevationFromMsl", + "totalDepthMdDriller", + "wellHeadProjected" + ] + }, + "name": { + "description": "The name of the CRS or the symbol/name of the unit.", + "title": "Name or Symbol", + "type": "string", + "example": [ + "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "ft" + ] + }, + "propertyValues": { + "description": "The list of property values, to which this meta data item provides Unit/CRS context to. Typically a unit symbol is a value to a data structure; this symbol is then registered in this propertyValues array and the persistableReference provides the absolute reference.", + "title": "Property Values", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "F", + "ftUS", + "deg" + ] + }, + "uncertainty": { + "description": "The uncertainty of the values measured given the unit or CRS unit.", + "title": "Uncertainty", + "type": "number" + } + }, + "required": [ + "kind", + "persistableReference" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractMetaItem.1.0.0.json" + }, + "opendes:osdu:AbstractLegalTags:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "Legal meta data like legal tags, relevant other countries, legal status.", + "additionalProperties": false, + "title": "Legal Meta Data", + "type": "object", + "properties": { + "legaltags": { + "description": "The list of legal tags.", + "title": "Legal Tags", + "type": "array", + "items": { + "type": "string" + } + }, + "otherRelevantDataCountries": { + "description": "The list of other relevant data countries.", + "title": "Other Relevant Data Countries", + "type": "array", + "items": { + "type": "string" + } + }, + "status": { + "description": "The legal status.", + "title": "Legal Status", + "type": "string" + } + }, + "required": [ + "legaltags", + "otherRelevantDataCountries" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalTags.1.0.0.json" + }, + "opendes:osdu:AbstractAccessControlList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "The access control tags associated with this entity.", + "additionalProperties": false, + "title": "Access Control List", + "type": "object", + "properties": { + "viewers": { + "description": "The list of viewers to which this data record is accessible/visible/discoverable.", + "title": "List of Viewers", + "type": "array", + "items": { + "type": "string" + } + }, + "owners": { + "description": "The list of owners of this data record.", + "title": "List of Owners", + "type": "array", + "items": { + "type": "string" + } + } + }, + "required": [ + "owners", + "viewers" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAccessControlList.1.0.0.json" + }, + "opendes:osdu:AbstractLegalParentList:1.0.0": { + "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.", + "$schema": "http://json-schema.org/draft-07/schema#", + "description": "A list of entity IDs in the data ecosystem, which act as legal parents to the current entity.", + "title": "Parent List", + "type": "object", + "properties": { + "parents": { + "description": "An array of none, one or many entity references in the data ecosystem, which identify the source of data in the legal sense. Example: the 'parents' will be queried when e.g. the subscription of source data services is terminated; access to the derivatives is also terminated.", + "title": "Parents", + "type": "array", + "items": { + "type": "string" + }, + "example": [] + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalParentList.1.0.0.json" + } + }, + "properties": { + "groupType": { + "const": "work-product", + "description": "The OSDU GroupType assigned to this resource object.", + "title": "Group Type" + }, + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/opendes:osdu:AbstractLegalParentList:1.0.0" + }, + "resourceCurationStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceCurationStatus:[^:]+:[0-9]*$", + "description": "Describes the current Curation status.", + "title": "Resource Curation Status", + "type": "string" + }, + "licenseState": { + "pattern": "^srn:opendes:reference-data\/LicenseState:[^:]+:[0-9]*$", + "description": "Indicates what kind of ownership Company has over data.", + "title": "License State", + "type": "string" + }, + "data": { + "allOf": [ + { + "type": "object", + "properties": { + "Description": { + "description": "Description of the purpose of the work product.", + "type": "string" + }, + "AuthorIDs": { + "description": "Array of Authors' names of the work product. Could be a person or company entity.", + "type": "array", + "items": { + "type": "string" + } + }, + "IsExtendedLoad": { + "description": "A flag that indicates if the work product is undergoing an extended load. It reflects the fact that the work product is in an early stage and may be updated before finalization.", + "type": "boolean" + }, + "Name": { + "description": "Name of the instance of Work Product - could be a shipment number.", + "type": "string" + }, + "Components": { + "type": "array", + "items": { + "pattern": "^srn:opendes:work-product-component\/[A-Za-z0-9]+:[^:]+:[0-9]*$", + "description": "The SRN which identifies this OSDU Work Product Component resource.", + "type": "string" + } + }, + "SpatialArea": { + "description": "A polygon boundary that reflects the locale of the content of the work product (location of the subject matter).", + "$ref": "opendes:osdu:AbstractSpatialLocation:1.0.0" + }, + "CreationDateTime": { + "format": "date-time", + "description": "Date that a resource (work product here) is formed outside of OSDU before loading (e.g. publication date, work product delivery package assembly date).", + "type": "string" + }, + "Annotations": { + "description": "Array of Annotations", + "type": "array", + "items": { + "type": "string" + } + }, + "BusinessActivities": { + "description": "Array of business processes/workflows that the work product has been through (ex. well planning, exploration).", + "type": "array", + "items": { + "description": "Business Activity", + "type": "string" + } + }, + "IsDiscoverable": { + "description": "A flag that indicates if the work product is searchable, which means covered in the search index.", + "type": "boolean" + }, + "SpatialPoint": { + "description": "A centroid point that reflects the locale of the content of the work product (location of the subject matter).", + "$ref": "opendes:osdu:AbstractSpatialLocation:1.0.0" + }, + "SubmitterName": { + "description": "Name of the person that first submitted the work product package to OSDU.", + "type": "string" + }, + "LineageAssertions": { + "description": "Defines relationships with other objects (any kind of Resource) upon which this work product depends. The assertion is directed only from the asserting WP to ancestor objects, not children. It should not be used to refer to files or artefacts within the WP -- the association within the WP is sufficient and Artefacts are actually children of the main WP file. They should be recorded in the Data.Artefacts[] array.", + "type": "array", + "items": { + "type": "object", + "title": "LineageAssertion", + "properties": { + "ID": { + "pattern": "^srn:opendes:[A-Za-z-]+\/[A-Za-z0-9]+:[^:]+:[0-9]*$", + "description": "The object reference identifying the DIRECT, INDIRECT, REFERENCE dependency.", + "type": "string" + }, + "LineageRelationshipType": { + "pattern": "^srn:opendes:reference-data\/LineageRelationshipType:[^:]+:[0-9]*$", + "description": "Used by LineageAssertion to describe the nature of the line of descent of a work product from a prior Resource, such as DIRECT, INDIRECT, REFERENCE. It is not for proximity (number of nodes away), it is not to cover all the relationships in a full ontology or graph, and it is not to describe the type of activity that created the asserting WP. LineageAssertion does not encompass a full provenance, process history, or activity model.", + "type": "string" + } + } + } + }, + "Tags": { + "description": "Array of key words to identify the work product, especially to help in search.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + { + "type": "object", + "properties": { + "ExtensionProperties": { + "type": "object", + "properties": {} + } + } + } + ] + }, + "kind": { + "pattern": "^[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[0-9]+.[0-9]+.[0-9]+$", + "description": "The schema identification for the OSDU resource object following the pattern <Namespace>:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.", + "title": "Entity Kind", + "type": "string", + "example": "namespace:osdu:WorkProduct:2.7.112" + }, + "ComponentsAssociativeIDs": { + "description": "ID of Files", + "type": "array", + "items": { + "type": "string" + } + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/opendes:osdu:AbstractAccessControlList:1.0.0" + }, + "source": { + "pattern": "^srn:opendes:master-data\/Organisation:[^:]+:[0-9]*$", + "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" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1831253916104085 + }, + "resourceVersionCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time when the current version of this resource entered the OSDU.", + "title": "Resource Version Creation DateTime", + "type": "string" + }, + "resourceHomeRegionID": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "description": "The name of the home [cloud environment] region for this OSDU resource object.", + "title": "Resource Home Region ID", + "type": "string" + }, + "resourceObjectCreationDateTime": { + "format": "date-time", + "description": "Timestamp of the time at which Version 1 of this OSDU resource object was originated.", + "title": "Resource Object Creation DateTime", + "type": "string" + }, + "resourceSecurityClassification": { + "pattern": "^srn:opendes:reference-data\/ResourceSecurityClassification:[^:]+:[0-9]*$", + "description": "Classifies the security level of the resource.", + "title": "Resource Security Classification", + "type": "string" + }, + "resourceHostRegionIDs": { + "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.", + "title": "Resource Host Region ID", + "type": "array", + "items": { + "pattern": "^srn:opendes:reference-data\/OSDURegion:[^:]+:[0-9]*$", + "type": "string" + } + }, + "resourceLifecycleStatus": { + "pattern": "^srn:opendes:reference-data\/ResourceLifecycleStatus:[^:]+:[0-9]*$", + "description": "Describes the current Resource Lifecycle status.", + "title": "Resource Lifecycle Status", + "type": "string" + }, + "legal": { + "description": "The entity's legal tags and compliance status.", + "title": "Legal Tags", + "$ref": "#/definitions/opendes:osdu:AbstractLegalTags:1.0.0" + }, + "id": { + "pattern": "^srn:opendes:work-product\/WorkProduct:[^:]+$", + "description": "The SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "srn:opendes:work-product/WorkProduct:146156b3-06aa-5195-b2f3-61c429f9f6ba" + }, + "existenceKind": { + "pattern": "^srn:opendes:reference-data\/ExistenceKind:[^:]+:[0-9]*$", + "description": "Where does this data resource sit in the cradle-to-grave span of its existence?", + "title": "Existence Kind", + "type": "string" + }, + "persistableReferences": { + "description": "The meta data section linking the 'unitKey', 'crsKey' to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/opendes:osdu:AbstractMetaItem:1.0.0" + } + } + }, + "required": [ + "kind", + "groupType", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/work-product/WorkProduct.1.0.0.json" +} diff --git a/tests/plugin-unit-tests/file_paths.py b/tests/plugin-unit-tests/file_paths.py new file mode 100644 index 0000000000000000000000000000000000000000..295877c85d0fe4adfce1621cb622a2fce1149d25 --- /dev/null +++ b/tests/plugin-unit-tests/file_paths.py @@ -0,0 +1,33 @@ +# 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_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/Wellbore.0.3.0.json" +SCHEMA_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/schema_Wellbore.3.0.0.json" +RECORD_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/record_Wellbore.0.3.0.json" + +MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/SeismicTraceData.json" +SCHEMA_FILE_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_File.1.0.0.json" +SCHEMA_WORK_PRODUCT_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_WorkProduct.1.0.0.json" +SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_SeismicTraceData.1.0.0.json" +RECORD_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/record_SeismicTraceData.json" + +MANIFEST_EMPTY_PATH = f"{DATA_PATH_PREFIX}/invalid/EmptyManifest.json" + +SEARCH_VALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseValid.json" +SEARCH_INVALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseInvalid.json" diff --git a/tests/plugin-unit-tests/mock_responses.py b/tests/plugin-unit-tests/mock_responses.py new file mode 100644 index 0000000000000000000000000000000000000000..aa5d0c5989cd25ce84558599737b5cfabd2c5178 --- /dev/null +++ b/tests/plugin-unit-tests/mock_responses.py @@ -0,0 +1,75 @@ +# 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 requsets' 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 + + +class MockSearchResponse(MockResponse): + + def __init__(self, body_path: str, status_code: http.HTTPStatus = 200, total_count: int = 0, + *args, **kwargs): + super().__init__(status_code) + self.body_path = body_path + self.total_count = total_count + + def json(self): + with open(self.body_path) as f: + response_dict = json.load(f) + if "totalCount" in response_dict: + response_dict["totalCount"] = self.total_count + return response_dict + + +class MockSchemaResponse(MockResponse): + + def __init__(self, schema_path: str, status_code=200, *args, **kwargs): + super().__init__(status_code) + self.schema_path = schema_path + + def json(self): + with open(self.schema_path) as f: + return json.load(f) diff --git a/tests/plugin-unit-tests/test_manifest_processor_r3.py b/tests/plugin-unit-tests/test_manifest_processor_r3.py new file mode 100644 index 0000000000000000000000000000000000000000..5a696615f32e6befba8c9f0d43cbefae40610a25 --- /dev/null +++ b/tests/plugin-unit-tests/test_manifest_processor_r3.py @@ -0,0 +1,249 @@ +# 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 + +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") + +from libs.context import Context +from libs.exceptions import EmptyManifestError +from deepdiff import DeepDiff +import pytest +import requests + +from file_paths import ( + MANIFEST_WELLBORE_VALID_PATH, + MANIFEST_EMPTY_PATH, + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + RECORD_WELLBORE_VALID_PATH, + RECORD_SEISMIC_TRACE_DATA_VALID_PATH, +) +from libs import process_manifest_r3 + +TENANT = "opendes" + + +class TestManifestProcessor: + + @staticmethod + def monkeypatch_storage_response( + monkeypatch, + response_content: bytes = b"{\"recordIds\": [\"test\"]}" + ): + """ + Make storage service request return mock response. + """ + + def mockresponse(url, data=None, **kwargs): + response = requests.Response() + response.status_code = http.HTTPStatus.OK + response._content = response_content + return response + + monkeypatch.setattr(requests, "put", mockresponse) + + @staticmethod + def monkeypatch_storage_response_error(monkeypatch, error_status: http.HTTPStatus): + """ + Make storage request return HTTPError response. + """ + + def mockresponse(url, data=None, **kwargs): + response = requests.Response() + response.status_code = error_status + response._content = b"{\"recordIds\": [\"test\"]}" + return response + + monkeypatch.setattr(requests, "put", mockresponse) + + @pytest.fixture(autouse=True) + def manifest_processor(self, monkeypatch, conf_path: str): + with open(conf_path) as f: + conf = json.load(f) + context = Context.populate(conf) + manifest_processor = process_manifest_r3.ManifestProcessor( + storage_url="", + dagrun_conf=conf, + context=context + ) + monkeypatch.setattr(manifest_processor, "generate_id", lambda manifest: "test_id") + return manifest_processor + + @pytest.fixture() + def mock_records_list(self, records_file_path: str): + """ + Mock records list from Storage service taken from file. + """ + with open(records_file_path) as f: + return json.load(f) + + @pytest.mark.parametrize( + "conf_path,records_file_path", + [ + pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + RECORD_SEISMIC_TRACE_DATA_VALID_PATH, + id="Valid WorkProduct"), + pytest.param(MANIFEST_WELLBORE_VALID_PATH, + RECORD_WELLBORE_VALID_PATH, + id="Valid Wellbore"), + ] + ) + def test_create_manifest_records( + self, + manifest_processor: process_manifest_r3.ManifestProcessor, + mock_records_list: list, + conf_path: str, + records_file_path: str + ): + actual_records = manifest_processor.create_manifest_records() + sort_predicate = lambda x: x["kind"] + actual_records.sort(key=sort_predicate) + mock_records_list.sort(key=sort_predicate) + assert actual_records == mock_records_list, DeepDiff(actual_records, mock_records_list, + ignore_order=True) + + @pytest.mark.parametrize( + "conf_path,records_file_path", + [ + pytest.param( + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + RECORD_SEISMIC_TRACE_DATA_VALID_PATH, + id="Valid WorkProduct" + ), + ] + ) + def test_save_record( + self, + monkeypatch, + manifest_processor: process_manifest_r3.ManifestProcessor, + mock_records_list: list, + conf_path: str, + records_file_path: str + ): + self.monkeypatch_storage_response(monkeypatch) + manifest_processor.save_record({}, records_file_path) + + @pytest.mark.parametrize( + "conf_path", + [ + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH + ] + ) + def test_save_record_invalid_storage_response_value( + self, + monkeypatch, + manifest_processor: process_manifest_r3.ManifestProcessor, + conf_path: str + ): + self.monkeypatch_storage_response(monkeypatch, b"{}") + with pytest.raises(ValueError): + manifest_processor.save_record({}, [{}]) + + @pytest.mark.parametrize( + "conf_path", + [ + pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, id="Valid WorkProduct") + ] + ) + def test_save_record_storage_response_http_error( + self, + monkeypatch, + manifest_processor: process_manifest_r3.ManifestProcessor, + conf_path: str + ): + self.monkeypatch_storage_response_error(monkeypatch, http.HTTPStatus.INTERNAL_SERVER_ERROR) + with pytest.raises(requests.HTTPError): + manifest_processor.save_record({}, conf_path) + + @pytest.mark.parametrize( + "conf_path", + [ + pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, id="WorkProduct"), + pytest.param(MANIFEST_WELLBORE_VALID_PATH, id="Master"), + + ] + ) + def test_process_manifest_valid( + self, + monkeypatch, + manifest_processor: process_manifest_r3.ManifestProcessor, + conf_path: str + ): + self.monkeypatch_storage_response(monkeypatch) + manifest_processor.process_manifest() + + @pytest.mark.parametrize( + "conf_path", + [ + pytest.param(MANIFEST_EMPTY_PATH, id="Empty Manifest"), + ] + ) + def test_process_empty_manifest( + self, + monkeypatch, + manifest_processor: process_manifest_r3.ManifestProcessor, + conf_path: str + ): + self.monkeypatch_storage_response(monkeypatch) + with pytest.raises(EmptyManifestError): + manifest_processor.process_manifest() + + @pytest.mark.parametrize( + "conf_path,expected_kind_name", + [ + pytest.param(MANIFEST_WELLBORE_VALID_PATH, "Wellbore", id="Valid Wellbore"), + ] + ) + def test_get_kind( + self, + monkeypatch, + manifest_processor: process_manifest_r3.ManifestProcessor, + conf_path: str, + expected_kind_name: str + ): + for manifest_part in manifest_processor.data_object["manifest"]: + kind = manifest_part["kind"] + assert expected_kind_name == manifest_processor._get_kind_name(kind) + + @pytest.mark.parametrize( + "conf_path,expected_kind_name", + [ + pytest.param(MANIFEST_WELLBORE_VALID_PATH, "Wellbore", id="Valid Wellbore"), + ] + ) + def test_generate_id( + self, + monkeypatch, + conf_path: str, + expected_kind_name: str + ): + with open(conf_path) as f: + conf = json.load(f) + context = process_manifest_r3.Context.populate(conf) + manifest_processor = process_manifest_r3.ManifestProcessor( + storage_url="", + dagrun_conf=conf, + context=context + ) + for manifest_part in manifest_processor.data_object["manifest"]: + group_type = manifest_part["groupType"] + kind_name = manifest_processor._get_kind_name(manifest_part["kind"]) + generated_id = manifest_processor.generate_id(manifest_part) + assert generated_id.startswith(f"{TENANT}:{group_type}_{kind_name}:") diff --git a/tests/plugin-unit-tests/test_operators_r3.py b/tests/plugin-unit-tests/test_operators_r3.py new file mode 100644 index 0000000000000000000000000000000000000000..e184a5ee80b3e10f7d0ae189727205ed75fa42de --- /dev/null +++ b/tests/plugin-unit-tests/test_operators_r3.py @@ -0,0 +1,143 @@ +# 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 libs.exceptions import PipelineFailedError +import pytest +import requests + +from file_paths import ( + MANIFEST_WELLBORE_VALID_PATH, + SEARCH_VALID_RESPONSE_PATH +) +from operators.process_manifest_r3 import ProcessManifestOperatorR3, SchemaValidator, \ + ManifestProcessor +from operators.search_record_id import SearchRecordIdOperator +from hooks.http_hooks import workflow_hook +from operators.update_status import UpdateStatusOperator +from mock_responses import MockSearchResponse, 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_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): + monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj: None) + monkeypatch.setattr(ManifestProcessor, "save_record", + lambda obj, headers, request_data: MockStorageResponse()) + task, context = self._create_task(ProcessManifestOperatorR3) + task.pre_execute(context) + task.execute(context) + + @pytest.mark.parametrize( + "record_ids,search_response_path", + [ + pytest.param( + ["test"], + SEARCH_VALID_RESPONSE_PATH + ), + pytest.param( + ["test", "test", "test"], + SEARCH_VALID_RESPONSE_PATH + ) + ] + ) + def test_search_id_operator(self, monkeypatch, record_ids: list, search_response_path: str): + # make validator pass + monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj: None) + task, context = self._create_task(SearchRecordIdOperator) + + monkeypatch.setattr(context["ti"], "xcom_pull", lambda **kwargs: record_ids) + mock_search_response = MockSearchResponse(body_path=search_response_path, + status_code=http.HTTPStatus.OK, + total_count=len(record_ids)) + monkeypatch.setattr(requests, "post", lambda *args, **kwargs: mock_search_response) + + 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, "post", 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/plugin-unit-tests/test_process_manifest_r3.py b/tests/plugin-unit-tests/test_process_manifest_r3.py deleted file mode 100644 index 8129f55048c598490520aa9e48f4ee9f99020431..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/test_process_manifest_r3.py +++ /dev/null @@ -1,69 +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 copy -import json -import os -import sys - -import libs.exceptions -import pytest - -sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") - -from data import process_manifest_r3 as test_data -from operators import process_manifest_r3 - - -@pytest.mark.parametrize( - "input", - [ - # pytest.param(test_data.CONF, id="Valid manifest"), - # pytest.param(test_data.EMPTY_MANIFEST, marks=pytest.mark.xfail, id="Empty manifest"), - pytest.param("data/workProduct/SeismicTraceData.json", id="Valid WorkProduct"), - pytest.param("data/master/Wellbore.0.3.0.json", id="Valid master") - ] -) -def test_create_manifest_records(input): - with open(input) as f: - conf = json.load(f) - context = process_manifest_r3.Context.populate(conf) - manifest_processor = process_manifest_r3.ManifestProcessor( - storage_url="", - dagrun_conf=conf, - context=context - ) - records = manifest_processor.create_manifest_records() - records = json.dumps(records, indent=4) - print(records) - - -@pytest.mark.parametrize( - "schema,manifest", - [ - pytest.param(test_data.TEST_SCHEMA, test_data.CONF, marks=pytest.mark.xfail, - id="Valid manifest"), - ] -) -def test_schema_validator(schema, manifest): - conf = copy.deepcopy(manifest) - context = process_manifest_r3.Context.populate(conf) - validator = process_manifest_r3.SchemaValidator( - "", - conf, - context - ) - validator._validate_schema(manifest["manifest"][0], schema) diff --git a/tests/plugin-unit-tests/test_refresh_token.py b/tests/plugin-unit-tests/test_refresh_token.py index b5e976a1ef4743c1f3dee3aa50a51dde80f6ae1a..d152d0175f3bf6c21b6074ac1d7e6e9c7e8b01e5 100644 --- a/tests/plugin-unit-tests/test_refresh_token.py +++ b/tests/plugin-unit-tests/test_refresh_token.py @@ -1,8 +1,25 @@ +# 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 os import sys -from unittest.mock import MagicMock import pytest +from google.oauth2 import service_account sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") @@ -10,41 +27,131 @@ from libs.refresh_token import AirflowTokenRefresher from libs.exceptions import SAFilePathError -def create_token_refresher() -> AirflowTokenRefresher: - token_refresher = AirflowTokenRefresher() - return token_refresher - - -@pytest.mark.parametrize( - "access_token", - [ - "test" - ] -) -def test_access_token_cached(access_token: str): - """ - Check if access token stored in Airflow Variables after refreshing it. - """ - token_refresher = create_token_refresher() - token_refresher.get_access_token_using_sa_file = MagicMock(return_value=access_token) - token_refresher.refresh_token() - assert token_refresher.airflow_variables.get("access_token") == access_token - - -@pytest.mark.parametrize( - "access_token", - [ - "test" - ] -) -def test_authorization_header(access_token: str): - token_refresher = create_token_refresher() - token_refresher.get_access_token_using_sa_file = MagicMock(return_value=access_token) - token_refresher.refresh_token() - assert token_refresher.authorization_header.get("Authorization") == f"Bearer {access_token}" - - -def test_raise_sa_path_error_on_getting_absent_sa_file(): - token_refresher = create_token_refresher() - with pytest.raises(SAFilePathError): - token_refresher.get_sa_file_info() +class MockCredentials: + + def __init__(self, access_token): + self.access_token = access_token + + def refresh(self, *args, **kwargs): + self.token = self.access_token + + +class TestAirflowTokenRefresher: + + @pytest.fixture() + def token_refresher(self) -> AirflowTokenRefresher: + token_refresher = AirflowTokenRefresher() + return token_refresher + + @pytest.fixture() + def mock_sa_file_info(self, monkeypatch, token_refresher: AirflowTokenRefresher, + sa_file_path: str): + """ + Return fake sa_file_info from fake path in environment. + """ + monkeypatch.setattr(os.environ, "get", lambda *args, **kwargs: sa_file_path) + monkeypatch.setattr(json, "load", lambda *args, **kwargs: "test") + monkeypatch.setattr(os.path, "isfile", lambda *args, **kwargs: True) + monkeypatch.setattr(token_refresher, "get_sa_info_from_file", + lambda *args, **kwargs: "test") + monkeypatch.setattr(token_refresher, "get_sa_info_from_google_storage", + lambda *args, **kwargs: "test") + + @pytest.fixture() + def mock_sa_credentials(self, monkeypatch, expected_token: str): + """ + Return fake service account credentials having expected token. + """ + monkeypatch.setattr(service_account.Credentials, + "from_service_account_info", + lambda *args, **kwargs: MockCredentials(expected_token)) + + @pytest.mark.parametrize( + "access_token", + [ + "test" + ] + ) + def test_access_token_cached(self, monkeypatch, token_refresher: AirflowTokenRefresher, + access_token: str): + """ + Check if access token stored in Airflow Variables after refreshing it. + """ + monkeypatch.setattr(token_refresher, "get_access_token_using_sa_file", + lambda *args, **kwargs: access_token) + token_refresher.refresh_token() + assert token_refresher.airflow_variables.get("access_token") == access_token + + @pytest.mark.parametrize( + "access_token", + [ + "test", + "aaaa" + ] + ) + def test_authorization_header(self, monkeypatch, token_refresher: AirflowTokenRefresher, + access_token: str): + """ + Check if Authorization header is 'Bearer <access_token>' + """ + monkeypatch.setattr(token_refresher, "get_access_token_using_sa_file", + lambda *args, **kwargs: access_token) + token_refresher.refresh_token() + assert token_refresher.authorization_header.get("Authorization") == f"Bearer {access_token}" + + def test_raise_sa_path_error_on_getting_absent_sa_file(self, + token_refresher: AirflowTokenRefresher): + """ + Check if error raises if sa file path is empty + """ + with pytest.raises(SAFilePathError): + token_refresher.get_sa_info() + + @pytest.mark.parametrize( + "expected_token, sa_file_path", + [ + pytest.param("test1", "/test", id="Local file"), + pytest.param("test2", "gs://test/test", id="GCS") + ] + ) + def test_refresh_token_using_sa_file( + self, + monkeypatch, + mock_sa_file_info, + mock_sa_credentials, + token_refresher: AirflowTokenRefresher, + sa_file_path: str, + expected_token: str, + ): + token_refresher.refresh_token() + assert token_refresher.access_token == expected_token + + @pytest.mark.parametrize( + "expected_token, sa_file_path", + [ + pytest.param("test1", "/test", id="Local file"), + pytest.param("test2", "gs://test/test", id="GCS") + ] + ) + def test_refresh_token_using_sa_file_no_cached_variable( + self, + monkeypatch, + mock_sa_file_info, + mock_sa_credentials, + token_refresher: AirflowTokenRefresher, + sa_file_path: str, + expected_token: str, + ): + """ + Check if token refreshes automatically if key is not stored in cache. + """ + + def mock_empty_variable(key): + """ + Raise error as if Airflow Variable don't have 'access_token' variable. + """ + raise KeyError + + monkeypatch.setattr(token_refresher.airflow_variables, "get", mock_empty_variable) + access_token = token_refresher.access_token + assert access_token == expected_token diff --git a/tests/plugin-unit-tests/test_schema_validator_r3.py b/tests/plugin-unit-tests/test_schema_validator_r3.py new file mode 100644 index 0000000000000000000000000000000000000000..ee258930cc2b626d4da5db18e73bd1a98a153f8d --- /dev/null +++ b/tests/plugin-unit-tests/test_schema_validator_r3.py @@ -0,0 +1,184 @@ +# 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 copy +import http +import requests +import json +import os +import sys + +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") + +from file_paths import ( + DATA_PATH_PREFIX, + MANIFEST_EMPTY_PATH, + SCHEMA_FILE_VALID_PATH, + SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH, + SCHEMA_WORK_PRODUCT_VALID_PATH, + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + MANIFEST_WELLBORE_VALID_PATH, + SCHEMA_WELLBORE_VALID_PATH +) +from mock_responses import MockSchemaResponse +from libs.context import Context +from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError +import pytest + +from libs.validate_schema import SchemaValidator + +TENANT = "opendes" + + +class TestSchemaValidator: + + @pytest.fixture + def schema_validator(self, monkeypatch, manifest_file: str, schema_file: str): + with open(manifest_file) as f: + manifest_file = json.load(f) + conf = copy.deepcopy(manifest_file) + context = Context.populate(conf) + validator = SchemaValidator( + "", + conf, + context + ) + if schema_file: + monkeypatch.setattr(requests, "get", + lambda *args, **kwargs: MockSchemaResponse(schema_file)) + return validator + + @pytest.mark.parametrize( + "manifest_file,schema_file", + [ + pytest.param( + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, None, + id="Valid manifest_file") + ] + ) + def test_schema_validator_validate_workproduct_manifest(self, + monkeypatch, + schema_validator: SchemaValidator, + manifest_file: str, + schema_file: str): + def mock_get_schema(uri: str): + if ":WorkProduct:" in uri: + schema_path = SCHEMA_WORK_PRODUCT_VALID_PATH + elif ":SeismicTraceData:" in uri: + schema_path = SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH + elif ":File:" in uri: + schema_path = SCHEMA_FILE_VALID_PATH + else: + print(uri) + raise + with open(schema_path) as f: + schema_content = json.load(f) + return schema_content + + monkeypatch.setattr(schema_validator, "get_schema_request", mock_get_schema) + schema_validator.validate_manifest() + + @pytest.mark.parametrize( + "manifest_file,schema_file", + [ + pytest.param( + MANIFEST_WELLBORE_VALID_PATH, + SCHEMA_WELLBORE_VALID_PATH, + id="Valid manifest Wellore"), + ] + ) + def test_schema_validator_master_manifest(self, monkeypatch, schema_validator, manifest_file, + schema_file): + def mock_get_schema(uri: str): + with open(schema_file) as f: + schema_content = json.load(f) + return schema_content + + monkeypatch.setattr(schema_validator, "get_schema_request", mock_get_schema) + schema_validator.validate_manifest() + + @pytest.mark.parametrize( + "manifest_file,schema_file", + [ + pytest.param( + MANIFEST_EMPTY_PATH, None, + id="Empty Manifest"), + ] + ) + def test_schema_validator_empty_manifest(self, + monkeypatch, + schema_validator: SchemaValidator, + manifest_file: str, + schema_file: str): + with pytest.raises(EmptyManifestError): + schema_validator.validate_manifest() + + @pytest.mark.parametrize( + "manifest_file,schema_file", + [ + pytest.param( + f"{DATA_PATH_PREFIX}/invalid/NotOsduFormat.json", None, + id="Not OSDU FORMAT"), + ] + ) + def test_schema_validator_not_osdu_format(self, + monkeypatch, + schema_validator: SchemaValidator, + manifest_file: str, + schema_file: str): + with pytest.raises(NotOSDUShemaFormatError): + schema_validator.validate_manifest() + + @pytest.mark.parametrize( + "manifest_file,schema_file,kind", + [ + pytest.param( + MANIFEST_WELLBORE_VALID_PATH, + SCHEMA_WELLBORE_VALID_PATH, + "opendes:osdu:Wellbore:0.3.0", + id="Valid manifest Wellore"), + ] + ) + def test_get_schema_request(self, + schema_validator: SchemaValidator, + manifest_file: str, + schema_file: str, + kind: str): + schema_validator.get_schema_request(kind) + + @pytest.mark.parametrize( + "manifest_file,schema_file,kind", + [ + pytest.param( + MANIFEST_WELLBORE_VALID_PATH, + SCHEMA_WELLBORE_VALID_PATH, + "opendes:osdu:Wellbore:0.3.0", + id="Valid manifest Wellore"), + ] + ) + def test_get_schema_error(self, + monkeypatch, + schema_validator: SchemaValidator, + manifest_file: str, + schema_file: str, + kind: str): + monkeypatch.setattr(requests, + "get", + lambda *args, **kwargs: MockSchemaResponse("{}", + http.HTTPStatus.INTERNAL_SERVER_ERROR)) + with pytest.raises(requests.HTTPError): + schema_validator.get_schema(kind) diff --git a/tests/plugin-unit-tests/test_search_record_id.py b/tests/plugin-unit-tests/test_search_record_id.py new file mode 100644 index 0000000000000000000000000000000000000000..63db7f5569468769984d02330b1345edd058133d --- /dev/null +++ b/tests/plugin-unit-tests/test_search_record_id.py @@ -0,0 +1,127 @@ +# 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 +import sys +import http +import requests +import pytest + +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") + +from file_paths import ( + SEARCH_INVALID_RESPONSE_PATH, + SEARCH_VALID_RESPONSE_PATH +) +from libs.exceptions import RecordsNotSearchableError +from libs.context import Context +from tenacity import stop_after_attempt +from libs.search_record_ids import SearchId +from mock_responses import MockSearchResponse + + +class TestManifestProcessor: + + def mock_storage_response( + self, + monkeypatch, + body_path: str, + status_code: http.HTTPStatus = http.HTTPStatus.OK, + total_count: int = 0 + ): + def mock_response(*args, **kwargs): + return MockSearchResponse(body_path, status_code, total_count) + + monkeypatch.setattr(requests, "post", mock_response) + # turn of retry for unit tests + SearchId.search_files.retry.stop = stop_after_attempt(1) + + @pytest.mark.parametrize( + "record_ids,search_response_path", + [ + pytest.param( + ["test"], + SEARCH_VALID_RESPONSE_PATH + ), + pytest.param( + ["test", "test", "test"], + SEARCH_VALID_RESPONSE_PATH + ) + ] + ) + def test_search_found_all_records(self, monkeypatch, record_ids: list, + search_response_path: str): + self.mock_storage_response(monkeypatch, search_response_path, total_count=len(record_ids)) + id_searcher = SearchId("http://test", record_ids, Context(app_key="", data_partition_id="")) + id_searcher.check_records_searchable() + + @pytest.mark.parametrize( + "record_ids,search_response_path", + [ + pytest.param( + ["test"], + SEARCH_VALID_RESPONSE_PATH + ), + pytest.param( + ["test", "test", "test"], + SEARCH_VALID_RESPONSE_PATH + ) + ] + ) + def test_search_not_found_all_records(self, monkeypatch, record_ids: list, + search_response_path: str): + invalid_total_count = len(record_ids) - 1 + self.mock_storage_response(monkeypatch, search_response_path, + total_count=invalid_total_count) + id_searcher = SearchId("", record_ids, Context(app_key="", data_partition_id="")) + with pytest.raises(RecordsNotSearchableError): + id_searcher.check_records_searchable() + + @pytest.mark.parametrize( + "record_ids,search_response_path", + [ + pytest.param( + ["test"], + SEARCH_INVALID_RESPONSE_PATH + ), + pytest.param( + ["test", "test", "test"], + SEARCH_INVALID_RESPONSE_PATH + ) + ] + ) + def test_search_got_wrong_response_value(self, monkeypatch, record_ids: list, + search_response_path: str): + self.mock_storage_response(monkeypatch, search_response_path) + id_searcher = SearchId("http://test", record_ids, Context(app_key="", data_partition_id="")) + with pytest.raises(ValueError): + id_searcher.check_records_searchable() + + @pytest.mark.parametrize( + "record_ids,search_response_path", + [ + pytest.param( + [], + SEARCH_INVALID_RESPONSE_PATH + ) + ] + ) + def test_searcher_got_no_record_ids(self, monkeypatch, record_ids: list, + search_response_path: str): + self.mock_storage_response(monkeypatch, search_response_path) + with pytest.raises(ValueError): + SearchId("http://test", record_ids, Context(app_key="", data_partition_id="")) diff --git a/tests/plugin-unit-tests/test_update_status_r3.py b/tests/plugin-unit-tests/test_update_status_r3.py new file mode 100644 index 0000000000000000000000000000000000000000..df05b36f845fa74e2e1fb89f4dcdd5a1599317b5 --- /dev/null +++ b/tests/plugin-unit-tests/test_update_status_r3.py @@ -0,0 +1,63 @@ +# 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 +import sys +import json +import http +import requests +import pytest + +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") + +from file_paths import ( + MANIFEST_WELLBORE_VALID_PATH +) +from libs.context import Context +from libs.update_status import UpdateStatus +from mock_responses import MockWorkflowResponse + + +class TestUpdateStatus: + + @pytest.fixture() + def status_updater(self, status: str, conf_path: str): + with open(conf_path) as f: + conf = json.load(f) + context = Context.populate(conf) + workflow_id = conf["WorkflowID"] + status_updater = UpdateStatus( + workflow_url = "http://test", + workflow_id=workflow_id, + context=context, + status=status + ) + return status_updater + + @pytest.mark.parametrize( + "conf_path,status", + [ + pytest.param( + MANIFEST_WELLBORE_VALID_PATH, + http.HTTPStatus.OK + ) + ] + ) + def test_update_workflow_status(self, monkeypatch, status_updater: UpdateStatus, conf_path: str, + status: str): + monkeypatch.setattr(requests, "post", lambda *args, **kwargs: MockWorkflowResponse()) + status_updater.update_workflow_status() diff --git a/tests/set_airflow_env.sh b/tests/set_airflow_env.sh index ea7a43213e04922c5ea6d23318e76a70135238ef..8b750c31184db33e83d44d25b876e0f41e1c6eaa 100755 --- a/tests/set_airflow_env.sh +++ b/tests/set_airflow_env.sh @@ -18,10 +18,13 @@ pip install dataclasses pip install jsonschema pip install google pip install google-cloud-storage +pip install deepdiff export ACL='{"viewers": ["foo"],"owners": ["foo"]}' export LEGAL='{"legaltags": ["foo"], "otherRelevantDataCountries": ["FR", "US", "CA"],"status": "compliant"}' -export WORKFLOW_URL="http://127.0.0.1:5000/wf" +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" +export SEARCH_URL="http://127.0.0.1:5000/sr/qr" export LOCALHOST="http://127.0.0.1:5000" export SEARCH_CONN_ID="http://127.0.0.1:5000" export WORKFLOW_CONN_ID="http://127.0.0.1:5000" @@ -45,14 +48,14 @@ airflow variables -s provider gcp airflow variables -s record_kind "odes:osdu:file:0.2.0" airflow variables -s schema_version "0.2.0" airflow variables -s workflow_url $WORKFLOW_URL -airflow variables -s update_status_ep wf/us -airflow variables -s search_url $LOCALHOST +airflow variables -s update_status_url $UPDATE_STATUS_URL +airflow variables -s search_url $SEARCH_URL +airflow variables -s schema_service_url $LOCALHOST airflow variables -s dataload_config_path $DATALOAD_CONFIG_PATH airflow variables -s search_query_ep sr/qr airflow variables -s access_token test airflow variables -s "sa-file-osdu" "test" -airflow connections -a --conn_id search --conn_uri $SEARCH_CONN_ID airflow connections -a --conn_id workflow --conn_uri $WORKFLOW_CONN_ID airflow connections -a --conn_id google_cloud_storage --conn_uri $WORKFLOW_CONN_ID