diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml index 62dc930e658d13db4bc7c0b3b43e8d84bf97540d..a533a36754aaa8b4765f9014e791e104a44d27da 100644 --- a/.gitlab-ci.yml +++ b/.gitlab-ci.yml @@ -61,6 +61,7 @@ unit_tests: - tests/./unit_tests.sh || EXIT_CODE=$? - exit ${EXIT_CODE} +# TODO: Think about how rsync must look. At the moment it looks messy. osdu-gcp-deploy: stage: deploy script: @@ -70,6 +71,7 @@ osdu-gcp-deploy: - gsutil -m rsync -d -R dags/libs $OSDU_GCP_DEPL_TARGET/dags/libs - gsutil -m rsync -x "dags/providers*" -d -R dags $OSDU_GCP_DEPL_TARGET/dags/ingestion - gsutil -m rsync -d -R dags/providers $OSDU_GCP_DEPL_TARGET/dags/providers + - gsutil cp dags/.airflowignore $OSDU_GCP_DEPL_TARGET/dags/ - gsutil -m rsync -R plugins $OSDU_GCP_DEPL_TARGET/plugins only: variables: diff --git a/src/dags/.airflowignore b/src/dags/.airflowignore new file mode 100644 index 0000000000000000000000000000000000000000..f227618f1f9141a89dbcabacf18a6dd009506c16 --- /dev/null +++ b/src/dags/.airflowignore @@ -0,0 +1,3 @@ +configs +libs +providers diff --git a/src/dags/libs/manifest_integrity.py b/src/dags/libs/manifest_integrity.py new file mode 100644 index 0000000000000000000000000000000000000000..345ee4f0bbed917c83149a42f896bdf49d460d82 --- /dev/null +++ b/src/dags/libs/manifest_integrity.py @@ -0,0 +1,157 @@ +# Copyright 2021 Google LLC +# Copyright 2021 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import copy +import json +import re +import logging +from typing import List, re as regex + +from libs.context import Context +from libs.exceptions import EmptyManifestError +from libs.search_record_ids import ExtendedSearchId + +logger = logging.getLogger() + + +class ManifestIntegrity(object): + """Class to validate if parents reference and master data are exists and + remove non-valid entities to provide integrity + """ + REFERENCE_DATA_ID_PATTERN = re.compile(r"[\w\-\.]+:reference-data\-\-[\w\-\.]+:.[^,;\"]+", + re.I + re.M) + MASTER_DATA_ID_PATTERN = re.compile(r"[\w\-\.]+:master-data\-\-[\w\-\.]+:.[^,;\"]+", + re.I + re.M) + WORK_PRODUCT_ID_PATTERN = re.compile(r"[\w\-\.]+:work-product\-\-[\w\-\.]+:.[^,;\"]+", + re.I + re.M) + WORK_PRODUCT_COMPONENT_ID_PATTERN = re.compile( + r"[\w\-\.]+:work-product-component\-\-[\w\-\.]+:.[^,;\"]+", re.I + re.M) + DATASET_ID_PATTERN = re.compile(r"[\w\-\.]+:dataset\-\-[\w\-\.]+:.[^,;\"]+", re.I + re.M) + + def __init__(self, search_url: str, token_refresher, context: Context): + self.search_url = search_url + self.token_refresher = token_refresher + self.context = context + self.ids_for_validation = [] + self.entities_ids = {} + self.patterns = [self.REFERENCE_DATA_ID_PATTERN, + self.MASTER_DATA_ID_PATTERN, + self.WORK_PRODUCT_ID_PATTERN, + self.WORK_PRODUCT_COMPONENT_ID_PATTERN, + self.DATASET_ID_PATTERN] + + def _match_id_with_pattern(self, pattern: regex.Pattern, source: str) -> List[str]: + return pattern.findall(source) + + def _collect_ids(self, manifest_file: dict): + """ + Collect manifest entities ids to exclude them while integrity checking + """ + entities_ids = [] + if manifest_file.get("ReferenceData"): + for elem in manifest_file["ReferenceData"]: + if elem.get("id"): + entities_ids.append(elem["id"]) + if manifest_file.get("MasterData"): + for elem in manifest_file["MasterData"]: + if elem.get("id"): + entities_ids.append(elem["id"]) + if manifest_file.get("Data"): + if manifest_file["Data"].get("WorkProduct") and manifest_file["Data"]["WorkProduct"].get("id"): + entities_ids.append(manifest_file["Data"]["WorkProduct"]["id"]) + if manifest_file["Data"].get("WorkProductComponents"): + for elem in manifest_file["Data"]["WorkProductComponents"]: + if elem.get("id"): + entities_ids.append(elem["id"]) + if manifest_file["Data"].get("Datasets"): + for elem in manifest_file["Data"]["Datasets"]: + if elem.get("id"): + entities_ids.append(elem["id"]) + self.entities_ids = set(entities_ids) + + def _extract_references(self, manifest: dict) -> List[str]: + """ + Extract reference ids from manifest + """ + temp_manifest_entity = copy.deepcopy(manifest) + temp_manifest_entity.pop("kind") + manifest_str = json.dumps(temp_manifest_entity) + ids_for_validation = [] + for pattern in self.patterns: + ids_for_validation.extend(self._match_id_with_pattern(pattern, manifest_str)) + ids_for_validation = list(set(ids_for_validation).difference(self.entities_ids)) + logger.debug(ids_for_validation) + return ids_for_validation + + def _validate_entity(self, manifest: dict) -> bool: + """ + Validate reference ids from manifest entity + """ + ids_for_validation = self._extract_references(manifest) + missing_ids = None + if ids_for_validation: + search_class = ExtendedSearchId(self.search_url, ids_for_validation, self.token_refresher, self.context) + found_ids, missing_ids = search_class.search_records() + if not missing_ids: + return True + return False + + def ensure_integrity(self, manifest_file: dict=None) -> dict: + """ + Validate reference ids in traversal manifest file + + :return: only valid entities in the same structure + """ + + if not manifest_file: + raise EmptyManifestError() + + self._collect_ids(manifest_file) + + if manifest_file.get("ReferenceData"): + for data in manifest_file["ReferenceData"]: + is_valid = self._validate_entity(data) + if not is_valid: + logger.warning(f"Resource with kind {data.get('kind')} was rejected") + manifest_file["ReferenceData"].remove(data) + if manifest_file.get("MasterData"): + master_data = manifest_file["MasterData"] + for data in master_data: + is_valid = self._validate_entity(data) + if not is_valid: + logger.warning(f"Resource with kind {data.get('kind')} was rejected") + manifest_file["MasterData"].remove(data) + if manifest_file.get("Data"): + if manifest_file["Data"].get("WorkProduct"): + work_product_data = manifest_file["Data"]["WorkProduct"] + is_valid = self._validate_entity(work_product_data) + if not is_valid: + logger.warning(f"Resource with kind {work_product_data.get('kind')} was rejected") + manifest_file["Data"]["WorkProduct"] = {} + if manifest_file["Data"].get("WorkProductComponents"): + wpc_data = manifest_file["Data"]["WorkProductComponents"] + for data in wpc_data: + is_valid = self._validate_entity(data) + if not is_valid: + logger.warning(f"Resource with kind {data.get('kind')} was rejected") + manifest_file["Data"]["WorkProductComponents"].remove(data) + if manifest_file["Data"].get("Datasets"): + datasets = manifest_file["Data"]["Datasets"] + for data in datasets: + is_valid = self._validate_entity(data) + if not is_valid: + logger.warning(f"Resource with kind {data.get('kind')} was rejected") + manifest_file["Data"]["Datasets"].remove(data) + return manifest_file diff --git a/src/dags/libs/processors/__init__.py b/src/dags/libs/processors/__init__.py new file mode 100644 index 0000000000000000000000000000000000000000..5511adb864013540d7eda3007c4ffb6186bc68cb --- /dev/null +++ b/src/dags/libs/processors/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2020 Google LLC +# Copyright 2020 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/src/dags/libs/processors/single_manifest_processor.py b/src/dags/libs/processors/single_manifest_processor.py new file mode 100644 index 0000000000000000000000000000000000000000..615b10b44b0b219f65e51e4f54bc5be81d7af4c7 --- /dev/null +++ b/src/dags/libs/processors/single_manifest_processor.py @@ -0,0 +1,107 @@ +# Copyright 2020 Google LLC +# Copyright 2020 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +""" +R3 Process Single Manifest helper. +""" + +import logging +from typing import List + +from libs.constants import FIRST_STORED_RECORD_INDEX +from libs.context import Context +from libs.manifest_analyzer import ManifestAnalyzer +from libs.refresh_token import TokenRefresher +from libs.process_manifest_r3 import ManifestProcessor +from libs.traverse_manifest import ManifestTraversal +from libs.validate_schema import SchemaValidator + +logger = logging.getLogger() + + +class SingleManifestProcessor(object): + + def __init__( + self, + storage_url: str, + payload_context: Context, + manifest_processor: ManifestProcessor, + schema_validator: SchemaValidator, + token_refresher: TokenRefresher, + ): + """Init SingleManifestProcessor.""" + self.storage_url = storage_url + self.payload_context = payload_context + self.manifest_processor = manifest_processor + self.schema_validator = schema_validator + self.token_refresher = token_refresher + + def _process_records(self, manifest_analyzer: ManifestAnalyzer) -> str: + """ + Process each entity from entity queue created according to child-parent relationships + between entities. + Replace surrogate-keys of parents inside child entities with system-generated keys. + + :param manifest_analyzer: Object with proper queue of entities + :yield: System-generated id of entity + """ + for entity in manifest_analyzer.entity_queue(): + try: + logger.debug(f"Process entity {entity}") + entity.replace_parents_surrogate_srns() + record_id = self.manifest_processor.process_manifest_records( + [entity.entity_info] + )[FIRST_STORED_RECORD_INDEX] + entity.system_srn = record_id + yield record_id + except Exception as e: + logger.warning(f"Can't process entity {entity}") + logger.error(e) + manifest_analyzer.add_unprocessed_entity(entity) + + def process_manifest(self, manifest: dict) -> List[str]: + """Execute manifest validation then process it. + + Execution steps: + 1) initialize schema validator + 2) validate manifest file against common schema + 3) traverse manifest file and extract manifest entities + 4) validate extracted manifest entities + 5) create an ingestion queue according to child-parent relationships between entities + 6) process valid manifest entities one-by-one + + :param context: Airflow context + :type context: dict + :return: List of record ids. + """ + record_ids = [] + manifest_schema = self.schema_validator.validate_common_schema(manifest) + traversal = ManifestTraversal(manifest, manifest_schema) + manifest_entities = traversal.traverse_manifest() + logger.debug(f"entities count: {len(manifest_entities)}") + valid_manifest_entities = self.schema_validator.validate_manifest(manifest_entities) + logger.debug(f"valid entities count: {len(valid_manifest_entities)}") + + manifest_analyzer = ManifestAnalyzer( + valid_manifest_entities, + self.storage_url, + self.token_refresher, + self.payload_context + ) + for record_id in self._process_records(manifest_analyzer): + record_ids.append(record_id) + logger.info(f"Processed ids {record_ids}") + return record_ids diff --git a/src/dags/libs/search_record_ids.py b/src/dags/libs/search_record_ids.py index f21620d7d997e011e9e9190b6672d3284ee60c42..7258c9a65ada27e9a31d9b9b452f529cc5716afb 100644 --- a/src/dags/libs/search_record_ids.py +++ b/src/dags/libs/search_record_ids.py @@ -17,9 +17,13 @@ import json import logging +from typing import List +from typing import Tuple import requests import tenacity +from requests import Response + from libs.context import Context from libs.exceptions import RecordsNotSearchableError from libs.mixins import HeadersMixin @@ -130,3 +134,54 @@ class SearchId(HeadersMixin): """Check if every record in self.record_ids is searchable.""" headers = self.request_headers self.search_files(headers) + + +class ExtendedSearchId(SearchId): + + def __init__(self, search_url: str, record_ids: list, token_refresher, context: Context): + super().__init__(search_url, record_ids, token_refresher, context) + + def _create_request_body(self): + """ + Create request body to send to Search service. + """ + query = self._create_search_query() + request_body = { + "kind": "*:*:*:*", + "query": query, + "returnedFields": ["id", "version"] + } + self.request_body = json.dumps(request_body) + + def _extract_id_from_response(self, response: dict): + results = response.get("results") + record_ids = [":".join([elem.get("id"), str(elem.get("version", ""))]) for elem in results] + record_ids.extend([elem.get("id") for elem in results]) + logger.debug(f"response ids: {record_ids}") + return record_ids + + @authorize() + def _make_post_request(self, headers: dict, request_body: dict) -> Response: + return requests.post(self.search_url, request_body, headers=headers) + + def search_records(self) -> Tuple[List, List]: + """ + Send request with recordIds to Search service. + """ + if self.request_body: + response = self._make_post_request(self.request_headers, self.request_body) + logger.debug(response.text) + data = response.json() + total_count = data.get('totalCount') + logger.debug(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}.") + response_records_ids = set(self._extract_id_from_response(data)) + if set(self.record_ids).issubset(response_records_ids): + return self.record_ids, [] + else: + found_ids = list(set(self.record_ids).intersection(response_records_ids)) + missing_ids = list(set(self.record_ids).difference(response_records_ids)) + logger.warning(f"The next ids are absent in the system: {missing_ids}") + return found_ids, missing_ids diff --git a/src/dags/libs/traverse_manifest.py b/src/dags/libs/traverse_manifest.py index 7ef4e401011b73bdbb30c479c364455856517164..4aaf49f1a1d91549fa6041b4c43cbf7193109648 100644 --- a/src/dags/libs/traverse_manifest.py +++ b/src/dags/libs/traverse_manifest.py @@ -37,8 +37,8 @@ class ManifestEntity: class ManifestTraversal(object): """Class to traverse manifest and extract all manifest records""" - def __init__(self, dagrun_conf: dict, manifest_schema: dict): - self.data_object = copy.deepcopy(dagrun_conf) + def __init__(self, manifest: dict, manifest_schema: dict): + self.manifest = copy.deepcopy(manifest) self.manifest_schema = manifest_schema self.manifest_info = [] @@ -70,30 +70,28 @@ class ManifestTraversal(object): :return: list of records """ - if "manifest" not in self.data_object: + if not self.manifest: raise EmptyManifestError - manifest_entities = [] - manifest_file = self.data_object["manifest"] - if manifest_file.get("ReferenceData"): - manifest_entities.extend(self._traverse_list(manifest_file["ReferenceData"], + if self.manifest.get("ReferenceData"): + manifest_entities.extend(self._traverse_list(self.manifest["ReferenceData"], "ReferenceData", self.manifest_schema["properties"])) - if manifest_file.get("MasterData"): - manifest_entities.extend(self._traverse_list(manifest_file["MasterData"], + if self.manifest.get("MasterData"): + manifest_entities.extend(self._traverse_list(self.manifest["MasterData"], "MasterData", self.manifest_schema["properties"])) - if manifest_file.get("Data"): - if manifest_file["Data"].get("WorkProduct"): + if self.manifest.get("Data"): + if self.manifest["Data"].get("WorkProduct"): manifest_entities.append(self._populate_manifest_entity( - manifest_file["Data"]["WorkProduct"], + self.manifest["Data"]["WorkProduct"], self.manifest_schema["properties"]["Data"]["properties"]["WorkProduct"]["$ref"])) - if manifest_file["Data"].get("WorkProductComponents"): - manifest_entities.extend(self._traverse_list(manifest_file["Data"]["WorkProductComponents"], + if self.manifest["Data"].get("WorkProductComponents"): + manifest_entities.extend(self._traverse_list(self.manifest["Data"]["WorkProductComponents"], "WorkProductComponents", self.manifest_schema["properties"]["Data"]["properties"])) - if manifest_file["Data"].get("Datasets"): - manifest_entities.extend(self._traverse_list(manifest_file["Data"]["Datasets"], + if self.manifest["Data"].get("Datasets"): + manifest_entities.extend(self._traverse_list(self.manifest["Data"]["Datasets"], "Datasets", self.manifest_schema["properties"]["Data"]["properties"])) return manifest_entities diff --git a/src/dags/libs/validate_schema.py b/src/dags/libs/validate_schema.py index 9006e40bcec39adcfa50179c6eb24caa7faa0a82..a58bd16f199a80cfb0c139bde846e3d3653077a4 100644 --- a/src/dags/libs/validate_schema.py +++ b/src/dags/libs/validate_schema.py @@ -106,22 +106,19 @@ class SchemaValidator(HeadersMixin): response = requests.get(uri, headers=headers, timeout=60) return response - def _delete_refs(self, schema_part: Union[dict, list]): + def _clear_data_fields(self, schema_part: Union[dict, list]): """ - Recursively clear a schema's object parts containing "$ref". + Clear a schema's ReferenceData, Data and MasterData fields". This method is used by generic manifest validation, deleting these fields make such a validation.more generic. :param schema_part: """ - if isinstance(schema_part, dict): - if "$ref" in schema_part: - schema_part.clear() - else: - for k in schema_part: - self._delete_refs(schema_part[k]) - elif isinstance(schema_part, list): - for i in schema_part: - self._delete_refs(i) + if schema_part.get("ReferenceData"): + schema_part["ReferenceData"] = {} + if schema_part.get("Data"): + schema_part["Data"] = {} + if schema_part.get("MasterData"): + schema_part["MasterData"] = {} def get_schema_request(self, uri: str) -> dict: """Get schema from Schema service. Change $id field to url. @@ -195,17 +192,15 @@ class SchemaValidator(HeadersMixin): :param manifest: :return: Manifest schema """ - if "manifest" not in manifest: - raise EmptyManifestError - schema = self.get_schema(manifest["manifest"]["kind"]) + schema = self.get_schema(manifest["kind"]) schema_without_refs = copy.deepcopy(schema) if schema_without_refs.get("properties"): - self._delete_refs(schema_without_refs["properties"]) + self._clear_data_fields(schema_without_refs["properties"]) else: - self._delete_refs(schema_without_refs) + self._clear_data_fields(schema_without_refs) logger.debug("Schema without refs") logger.debug(f"{schema_without_refs}") - self._validate_against_schema(schema, manifest) + self._validate_against_schema(schema_without_refs, manifest) return schema def validate_manifest(self, manifest_records: List[ManifestEntity]) -> List[ManifestEntity]: @@ -215,8 +210,6 @@ class SchemaValidator(HeadersMixin): :return: List of entities passed the validation """ validated_records = [] - if not manifest_records: - raise EmptyManifestError for manifest_record in manifest_records: manifest = manifest_record.entity if isinstance(manifest, dict) and manifest.get("kind"): diff --git a/src/dags/osdu-ingest-r3.py b/src/dags/osdu-ingest-r3.py index 1177305dc5fdd6860e243c77025595a9205ac7e6..32323f869a8902ecc31548f072daefe96bfa17ef 100644 --- a/src/dags/osdu-ingest-r3.py +++ b/src/dags/osdu-ingest-r3.py @@ -19,9 +19,19 @@ from datetime import timedelta import airflow from airflow import DAG +from airflow.models import Variable +from airflow.operators.dummy_operator import DummyOperator +from airflow.operators.python_operator import BranchPythonOperator +from libs.exceptions import NotOSDUSchemaFormatError +from operators.ensure_manifest_integrity import EnsureManifestIntegrityOperator from operators.process_manifest_r3 import ProcessManifestOperatorR3 from operators.update_status import UpdateStatusOperator +BATCH_NUMBER = int(Variable.get("batch_count", "3")) +PROCESS_SINGLE_MANIFEST_FILE = "process_single_manifest_file_task" +PROCESS_BATCH_MANIFEST_FILE = "batch_upload" +ENSURE_INTEGRITY_TASK = "provide_manifest_integrity_task" + default_args = { "start_date": airflow.utils.dates.days_ago(0), "retries": 0, @@ -30,29 +40,78 @@ default_args = { } workflow_name = "Osdu_ingest" -dag = DAG( + + +def is_batch(**context): + """ + :param context: Dag context + :return: SubDag to be executed next depending on Manifest type + """ + manifest = context["dag_run"].conf["execution_context"].get("manifest") + + if isinstance(manifest, dict): + subdag = ENSURE_INTEGRITY_TASK + elif isinstance(manifest, list): + subdag = PROCESS_BATCH_MANIFEST_FILE + else: + raise NotOSDUSchemaFormatError(f"Manifest must be either 'dict' or 'list'. " + f"Got {manifest}.") + return subdag + + +with DAG( workflow_name, default_args=default_args, - description="liveness monitoring dag", + description="R3 manifest processing with providing integrity", schedule_interval=None, dagrun_timeout=timedelta(minutes=60) -) - -update_status_running_op = UpdateStatusOperator( - task_id="update_status_running_task", - dag=dag -) - -update_status_finished_op = UpdateStatusOperator( - task_id="update_status_finished_task", - dag=dag, - trigger_rule="all_done", -) - -process_manifest_op = ProcessManifestOperatorR3( - task_id="proccess_manifest_task", - provide_context=True, - dag=dag -) - -update_status_running_op >> process_manifest_op >> update_status_finished_op # pylint: disable=pointless-statement +) as dag: + update_status_running_op = UpdateStatusOperator( + task_id="update_status_running_task", + dag=dag + ) + + branch_is_batch_op = BranchPythonOperator( + task_id="check_payload_type", + dag=dag, + provide_context=True, + python_callable=is_batch + ) + + update_status_finished_op = UpdateStatusOperator( + task_id="update_status_finished_task", + dag=dag, + trigger_rule="all_done", + ) + + process_single_manifest_file = ProcessManifestOperatorR3( + task_id=PROCESS_SINGLE_MANIFEST_FILE, + provide_context=True, + dag=dag, + previous_task_id=f"provide_manifest_integrity_task" + ) + + ensure_integrity_op = EnsureManifestIntegrityOperator( + task_id=ENSURE_INTEGRITY_TASK, + provide_context=True, + dag=dag + ) + + # Dummy operator as entry point into parallel task of batch upload + batch_upload = DummyOperator( + dag=dag, + task_id=PROCESS_BATCH_MANIFEST_FILE + ) + + for batch in range(0, BATCH_NUMBER): + batch_upload >> ProcessManifestOperatorR3( + task_id=f"process_manifest_task_{batch + 1}", + provide_context=True, + dag=dag, + previous_task_id=f"provide_manifest_integrity_task_{batch + 1}", + batch_number=batch + 1 + ) >> update_status_finished_op + +update_status_running_op >> branch_is_batch_op # pylint: disable=pointless-statement +branch_is_batch_op >> batch_upload # pylint: disable=pointless-statement +branch_is_batch_op >> ensure_integrity_op >> process_single_manifest_file >> update_status_finished_op # pylint: disable=pointless-statement diff --git a/src/plugins/operators/ensure_manifest_integrity.py b/src/plugins/operators/ensure_manifest_integrity.py new file mode 100644 index 0000000000000000000000000000000000000000..5adaf978062f2f420fb1ccefe32e9f4d2090cc53 --- /dev/null +++ b/src/plugins/operators/ensure_manifest_integrity.py @@ -0,0 +1,60 @@ +# Copyright 2021 Google LLC +# Copyright 2021 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +"""R3 Validate reference Manifest operator.""" +import logging + +from airflow.utils import apply_defaults +from airflow.models import BaseOperator, Variable +from libs.context import Context +from libs.manifest_integrity import ManifestIntegrity +from libs.refresh_token import AirflowTokenRefresher + + +logger = logging.getLogger() + + +class EnsureManifestIntegrityOperator(BaseOperator): + """Operator to validate ref inside manifest R3 and remove invalid entities.""" + + ui_color = '#dad5ff' + ui_fgcolor = '#000000' + + @apply_defaults + def __init__(self, + *args, **kwargs): + """Init base operator and obtain base urls from Airflow Variables.""" + super().__init__(*args, **kwargs) + + self.search_url = Variable.get('search_url') + + def execute(self, context: dict): + """Execute manifest validation then process it. + + :param context: Airflow context + :type context: dict + """ + payload_context = Context.populate(context["dag_run"].conf["execution_context"]) + token_refresher = AirflowTokenRefresher() + + manifest_integrity = ManifestIntegrity( + self.search_url, + token_refresher, + payload_context, + ) + valid_manifest_entities = manifest_integrity.ensure_integrity( + context["dag_run"].conf["execution_context"].get("manifest")) + logger.debug(valid_manifest_entities) + return {"manifest": valid_manifest_entities} diff --git a/src/plugins/operators/process_manifest_r3.py b/src/plugins/operators/process_manifest_r3.py index 01ee3392d919cce8a7274469b05e3846f8dae781..6ddebab7c6116b53b1a46717b447609a9ce8552b 100644 --- a/src/plugins/operators/process_manifest_r3.py +++ b/src/plugins/operators/process_manifest_r3.py @@ -19,21 +19,27 @@ R3 Process Manifest operator. """ import logging +from math import ceil +from typing import List, Tuple, TypeVar from airflow.utils import apply_defaults from airflow.models import BaseOperator, Variable -from libs.constants import FIRST_STORED_RECORD_INDEX +from jsonschema import SchemaError from libs.context import Context -from libs.manifest_analyzer import ManifestAnalyzer +from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError, \ + UploadFileError, GetSchemaError from libs.source_file_check import SourceFileChecker from libs.handle_file import FileHandler from libs.refresh_token import AirflowTokenRefresher from libs.process_manifest_r3 import ManifestProcessor -from libs.traverse_manifest import ManifestTraversal from libs.validate_schema import SchemaValidator +from libs.processors.single_manifest_processor import SingleManifestProcessor +from requests import HTTPError logger = logging.getLogger() +ManifestType = TypeVar("ManifestType", dict, List[dict]) + class ProcessManifestOperatorR3(BaseOperator): """Operator to process manifest R3.""" @@ -42,51 +48,88 @@ class ProcessManifestOperatorR3(BaseOperator): ui_fgcolor = '#000000' @apply_defaults - def __init__(self, *args, **kwargs): + def __init__(self, previous_task_id: str = None, batch_number=3, *args, **kwargs): """Init base operator and obtain base urls from Airflow Variables.""" super().__init__(*args, **kwargs) + self.previous_task_id = previous_task_id + self.batch_number = batch_number self.schema_service_url = Variable.get('schema_service_url') self.storage_url = Variable.get('storage_url') self.file_service_url = Variable.get('file_service_url') - self.processed_entities = [] - - def _process_records(self, manifest_analyzer: ManifestAnalyzer, - manifest_processor: ManifestProcessor) -> str: - for entity in manifest_analyzer.entity_queue(): - try: - logger.debug(f"Process entity {entity}") - entity.replace_parents_surrogate_srns() - record_id = manifest_processor.process_manifest_records( - [entity.entity_info] - )[FIRST_STORED_RECORD_INDEX] - entity.system_srn = record_id - yield record_id - self.processed_entities.append(f"{entity.srn}:{entity.system_srn}") - except Exception as e: - logger.warning(f"Can't process entity {entity}") - logger.error(e) - manifest_analyzer.add_unprocessed_entity(entity) + self.batch_count = int(Variable.get("batch_count", "3")) + + def _get_manifest_files_range(self, manifests: List[dict]) -> Tuple[int, int]: + """ + Get start and end indexes of a manifest files slice to be processed within this task. + :param manifests: A list of manifests + :return: start index, end index + """ + split_size = ceil(len(manifests) / self.batch_count) + slice_start_index = (self.batch_number - 1) * split_size + slice_end_index = self.batch_number * split_size + return slice_start_index, slice_end_index + + def _process_manifest( + self, + single_manifest_processor: SingleManifestProcessor, + manifest: ManifestType + ) -> List[str]: + """ + + :param single_manifest_processor: Object to process a single manifest file. + Processing includes validation against schemas, storing records enc. + :param manifest: A single manifest file or a list of them. + :return: + """ + if isinstance(manifest, dict): + record_ids = single_manifest_processor.process_manifest(manifest) + + elif isinstance(manifest, list): + record_ids = [] + slice_start_index, slice_end_index = self._get_manifest_files_range(manifest) + logger.debug(f"Start and indexes {slice_start_index}:{slice_end_index}") + for single_manifest in manifest[slice_start_index:slice_end_index]: + logger.debug(f"processing {single_manifest}") + try: + record_ids.extend(single_manifest_processor.process_manifest(single_manifest)) + except (UploadFileError, HTTPError, GetSchemaError, SchemaError) as e: + logger.error(f"Can't process {single_manifest}") + logger.error(e) + continue + else: + raise NotOSDUSchemaFormatError( + f"Manifest {manifest} must be either not empty 'list' or 'dict'") + + return record_ids + + def _get_manifest_data(self, context: dict, execution_context: dict) -> ManifestType: + if self.previous_task_id: + previous_task_value = context["ti"].xcom_pull(task_ids=self.previous_task_id, + key="return_value") + if previous_task_value: + manifest_data = previous_task_value["manifest"] + else: + manifest_data = execution_context["manifest"] + else: + manifest_data = execution_context["manifest"] + return manifest_data def execute(self, context: dict): """Execute manifest validation then process it. - Execution steps: - 1) initialize schema validator - 2) validate manifest file against common schema - 3) traverse manifest file and extract manifest entities - 4) validate extracted manifest entities - 5) process valid manifest entities + Get a single manifest file or a list of them. + If it is a list, calculate which range (slice) of manifest files must be processed and then + process this range one by one. :param context: Airflow context :type context: dict """ - record_ids = [] execution_context = context["dag_run"].conf["execution_context"] - payload_context = Context.populate(execution_context) token_refresher = AirflowTokenRefresher() file_handler = FileHandler(self.file_service_url, token_refresher, payload_context) source_file_checker = SourceFileChecker() + manifest_processor = ManifestProcessor( storage_url=self.storage_url, file_handler=file_handler, @@ -99,24 +142,20 @@ class ProcessManifestOperatorR3(BaseOperator): token_refresher, payload_context ) - - manifest_schema = validator.validate_common_schema(execution_context) - traversal = ManifestTraversal(execution_context, manifest_schema) - manifest_entities = traversal.traverse_manifest() - logger.debug(f"entities count: {len(manifest_entities)}") - valid_manifest_entities = validator.validate_manifest(manifest_entities) - logger.debug(f"valid entities count: {len(valid_manifest_entities)}") - - manifest_analyzer = ManifestAnalyzer( - valid_manifest_entities, - self.storage_url, - token_refresher, - payload_context + single_manifest_processor = SingleManifestProcessor( + storage_url=self.storage_url, + payload_context=payload_context, + manifest_processor=manifest_processor, + schema_validator=validator, + token_refresher=token_refresher, ) - for record_id in self._process_records(manifest_analyzer, manifest_processor): - record_ids.append(record_id) - logger.info(f"Surrogate-key:system-generated-id list {self.processed_entities}") - logger.info(f"Processed ids {record_ids}") + manifest_data = self._get_manifest_data(context, execution_context) + logger.debug(f"Manifest data: {manifest_data}") + if not manifest_data: + raise EmptyManifestError( + f"Data {context['dag_run'].conf} doesn't contain 'manifest field'") + record_ids = self._process_manifest(single_manifest_processor, manifest_data) + logger.info(f"Processed ids {record_ids}") context["ti"].xcom_push(key="record_ids", value=record_ids) diff --git a/tests/plugin-unit-tests/data/manifests/new_schema_Manifest.1.0.0.json b/tests/plugin-unit-tests/data/manifests/new_schema_Manifest.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..2ccfa2e9ab4230361c69760731398d39f1a2f148 --- /dev/null +++ b/tests/plugin-unit-tests/data/manifests/new_schema_Manifest.1.0.0.json @@ -0,0 +1,2403 @@ +{ + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:Manifest:1.0.0", + "description": "Load manifest applicable for all types defined as 'kind', i.e. registered as schemas with the Schema Service. It supports loading of individual 'records' of any group-type or combinations. The load sequence follows a well-defined sequence. The 'ReferenceData' array is processed first (if populated). The 'MasterData' array is processed second (if populated) second. The 'Data' structure is processed last (if populated). Inside the 'Data' property the 'Datasets' array is processed first, followed by the 'WorkProductComponents' array, the 'WorkProduct' is processed last. Any arrays are ordered. should there be interdependencies, the dependent items must be placed behind their relationship targets, e.g. a master-data Well record must placed in the 'MasterData' array before its Wellbores.", + "title": "Load Manifest Schema", + "type": "object", + "definitions": { + "osdu:wks:AbstractCommonResources:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractCommonResources:1.0.0", + "description": "Common resources to be injected at root 'data' level for every entity, which is persistable in Storage. The insertion is performed by the OsduSchemaComposer script.", + "title": "OSDU Common Resources", + "type": "object", + "properties": { + "ResourceHomeRegionID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-OSDURegion:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "The name of the home [cloud environment] region for this OSDU resource object.", + "x-osdu-relationship": [ + { + "EntityType": "OSDURegion", + "GroupType": "reference-data" + } + ], + "title": "Resource Home Region ID", + "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": { + "x-osdu-relationship": [ + { + "EntityType": "OSDURegion", + "GroupType": "reference-data" + } + ], + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-OSDURegion:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "type": "string" + } + }, + "ResourceLifecycleStatus": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ResourceLifecycleStatus:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "Describes the current Resource Lifecycle status.", + "x-osdu-relationship": [ + { + "EntityType": "ResourceLifecycleStatus", + "GroupType": "reference-data" + } + ], + "title": "Resource Lifecycle Status", + "type": "string" + }, + "ResourceSecurityClassification": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ResourceSecurityClassification:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "Classifies the security level of the resource.", + "x-osdu-relationship": [ + { + "EntityType": "ResourceSecurityClassification", + "GroupType": "reference-data" + } + ], + "title": "Resource Security Classification", + "type": "string" + }, + "ResourceCurationStatus": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ResourceCurationStatus:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "Describes the current Curation status.", + "x-osdu-relationship": [ + { + "EntityType": "ResourceCurationStatus", + "GroupType": "reference-data" + } + ], + "title": "Resource Curation Status", + "type": "string" + }, + "ExistenceKind": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ExistenceKind:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "Where does this data resource sit in the cradle-to-grave span of its existence?", + "x-osdu-relationship": [ + { + "EntityType": "ExistenceKind", + "GroupType": "reference-data" + } + ], + "title": "Existence Kind", + "type": "string" + }, + "Source": { + "title": "Data Source", + "type": "string", + "description": "The entity that produced the record, or from which it is received; could be an organization, agency, system, internal team, or individual. For informational purposes only, the list of sources is not governed." + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractCommonResources.1.0.0.json" + }, + "osdu:wks:AbstractMetaItem:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "oneOf": [ + { + "title": "FrameOfReferenceUOM", + "type": "object", + "properties": { + "name": { + "description": "The unit symbol or name of the unit.", + "title": "UOM Unit Symbol", + "type": "string", + "example": "ft[US]" + }, + "persistableReference": { + "description": "The self-contained, persistable reference string uniquely identifying the Unit.", + "title": "UOM Persistable Reference", + "type": "string", + "example": "{\"abcd\":{\"a\":0.0,\"b\":1200.0,\"c\":3937.0,\"d\":0.0},\"symbol\":\"ft[US]\",\"baseMeasurement\":{\"ancestry\":\"L\",\"type\":\"UM\"},\"type\":\"UAD\"}" + }, + "unitOfMeasureID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-UnitOfMeasure:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "SRN to unit of measure reference.", + "x-osdu-relationship": [ + { + "EntityType": "UnitOfMeasure", + "GroupType": "reference-data" + } + ], + "type": "string", + "example": "namespace:reference-data--UnitOfMeasure:ftUS:" + }, + "kind": { + "const": "Unit", + "title": "UOM Reference Kind", + "description": "The kind of reference, 'Unit' for FrameOfReferenceUOM." + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides Unit 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": "UOM Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "HorizontalDeflection.EastWest", + "HorizontalDeflection.NorthSouth" + ] + } + }, + "required": [ + "kind", + "persistableReference" + ] + }, + { + "title": "FrameOfReferenceCRS", + "type": "object", + "properties": { + "name": { + "description": "The name of the CRS.", + "title": "CRS Name", + "type": "string", + "example": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]" + }, + "coordinateReferenceSystemID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-CoordinateReferenceSystem:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "SRN to CRS reference.", + "x-osdu-relationship": [ + { + "EntityType": "CoordinateReferenceSystem", + "GroupType": "reference-data" + } + ], + "type": "string", + "example": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:" + }, + "persistableReference": { + "description": "The self-contained, persistable reference string uniquely identifying the CRS.", + "title": "CRS Persistable Reference", + "type": "string", + "example": "{\"authCode\":{\"auth\":\"EPSG\",\"code\":\"32615\"},\"type\":\"LBC\",\"ver\":\"PE_10_3_1\",\"name\":\"WGS_1984_UTM_Zone_15N\",\"wkt\":\"PROJCS[\\\"WGS_1984_UTM_Zone_15N\\\",GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Transverse_Mercator\\\"],PARAMETER[\\\"False_Easting\\\",500000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",-93.0],PARAMETER[\\\"Scale_Factor\\\",0.9996],PARAMETER[\\\"Latitude_Of_Origin\\\",0.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",32615]]\"}" + }, + "kind": { + "const": "CRS", + "title": "CRS Reference Kind", + "description": "The kind of reference, constant 'CRS' for FrameOfReferenceCRS." + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides 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": "CRS Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + }, + "required": [ + "kind", + "persistableReference" + ] + }, + { + "title": "FrameOfReferenceDateTime", + "type": "object", + "properties": { + "name": { + "description": "The name of the DateTime format and reference.", + "title": "DateTime Name", + "type": "string", + "example": "UTC" + }, + "persistableReference": { + "description": "The self-contained, persistable reference string uniquely identifying DateTime reference.", + "title": "DateTime Persistable Reference", + "type": "string", + "example": "{\"format\":\"yyyy-MM-ddTHH:mm:ssZ\",\"timeZone\":\"UTC\",\"type\":\"DTM\"}" + }, + "kind": { + "const": "DateTime", + "title": "DateTime Reference Kind", + "description": "The kind of reference, constant 'DateTime', for FrameOfReferenceDateTime." + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides DateTime 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": "DateTime Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "Acquisition.StartTime", + "Acquisition.EndTime" + ] + } + }, + "required": [ + "kind", + "persistableReference" + ] + }, + { + "title": "FrameOfReferenceAzimuthReference", + "type": "object", + "properties": { + "name": { + "description": "The name of the CRS or the symbol/name of the unit.", + "title": "AzimuthReference Name", + "type": "string", + "example": "TrueNorth" + }, + "persistableReference": { + "description": "The self-contained, persistable reference string uniquely identifying AzimuthReference.", + "title": "AzimuthReference Persistable Reference", + "type": "string", + "example": "{\"code\":\"TrueNorth\",\"type\":\"AZR\"}" + }, + "kind": { + "const": "AzimuthReference", + "title": "AzimuthReference Reference Kind", + "description": "The kind of reference, constant 'AzimuthReference', for FrameOfReferenceAzimuthReference." + }, + "propertyNames": { + "description": "The list of property names, to which this meta data item provides AzimuthReference 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": "AzimuthReference Property Names", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "Bearing" + ] + } + }, + "required": [ + "kind", + "persistableReference" + ] + } + ], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractMetaItem:1.0.0", + "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", + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractMetaItem.1.0.0.json" + }, + "osdu:wks:AbstractLegalParentList:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractLegalParentList:1.0.0", + "description": "A list of entity IDs in the data ecosystem, which act as legal parents to the current entity. This structure is included by the SystemProperties \"ancestry\", which is part of all OSDU records. Not extensible.", + "additionalProperties": false, + "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. In contract to other relationships, the source record version is required. 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": { + "x-osdu-relationship": [], + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]+$", + "type": "string" + }, + "example": [] + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalParentList.1.0.0.json" + }, + "osdu:wks:reference-data--GenericReferenceData:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:reference-data--GenericReferenceData:1.0.0", + "description": "An auto-generated placeholder schema representing reference-data group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.", + "title": "GenericReferenceData", + "type": "object", + "properties": { + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0" + }, + "data": { + "allOf": [ + { + "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0" + } + ] + }, + "kind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[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": "osdu:wks:reference-data--GenericReferenceData:1.0.0" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1562066009929332 + }, + "tags": { + "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.", + "additionalProperties": { + "type": "string" + }, + "title": "Tag Dictionary", + "type": "object", + "example": { + "NameOfKey": "String value" + } + }, + "modifyUser": { + "description": "The user reference, which created this version of this resource object. Set by the System.", + "title": "Resource Object Version Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "modifyTime": { + "format": "date-time", + "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Version Creation DateTime", + "type": "string", + "example": "2020-12-16T11:52:24.477Z" + }, + "createTime": { + "format": "date-time", + "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Creation DateTime", + "type": "string", + "example": "2020-12-16T11:46:20.163Z" + }, + "meta": { + "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0" + } + }, + "legal": { + "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.", + "title": "Legal Tags", + "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0" + }, + "createUser": { + "description": "The user reference, which created the first version of this resource object. Set by the System.", + "title": "Resource Object Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "id": { + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:reference-data\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$", + "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "namespace:reference-data--GenericReferenceData:63ca0ed3-d6fb-53f0-8549-0916ef144266" + } + }, + "required": [ + "id", + "kind", + "acl", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/reference-data/GenericReferenceData.1.0.0.json" + }, + "osdu:wks:AbstractFeatureCollection:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractFeatureCollection:1.0.0", + "description": "GeoJSON feature collection as originally published in https://geojson.org/schema/FeatureCollection.json. Attention: the coordinate order is fixed: Longitude first, followed by Latitude, optionally height above MSL (EPSG:5714) as third coordinate.", + "title": "GeoJSON FeatureCollection", + "type": "object", + "required": [ + "type", + "features" + ], + "properties": { + "type": { + "type": "string", + "enum": [ + "FeatureCollection" + ] + }, + "features": { + "type": "array", + "items": { + "title": "GeoJSON Feature", + "type": "object", + "required": [ + "type", + "properties", + "geometry" + ], + "properties": { + "geometry": { + "oneOf": [ + { + "type": "null" + }, + { + "title": "GeoJSON Point", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + }, + "type": { + "type": "string", + "enum": [ + "Point" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON LineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "LineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON Polygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "Polygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON MultiPoint", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "MultiPoint" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON MultiLineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "MultiLineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON MultiPolygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "MultiPolygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON GeometryCollection", + "type": "object", + "required": [ + "type", + "geometries" + ], + "properties": { + "type": { + "type": "string", + "enum": [ + "GeometryCollection" + ] + }, + "geometries": { + "type": "array", + "items": { + "oneOf": [ + { + "title": "GeoJSON Point", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + }, + "type": { + "type": "string", + "enum": [ + "Point" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON LineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "LineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON Polygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "Polygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON MultiPoint", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "MultiPoint" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON MultiLineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "MultiLineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "GeoJSON MultiPolygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "MultiPolygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + } + ] + } + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + } + ] + }, + "type": { + "type": "string", + "enum": [ + "Feature" + ] + }, + "properties": { + "oneOf": [ + { + "type": "null" + }, + { + "type": "object" + } + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + } + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractFeatureCollection.1.0.0.json" + }, + "osdu:wks:AbstractAnyCrsFeatureCollection:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractAnyCrsFeatureCollection:1.0.0", + "description": "A schema like GeoJSON FeatureCollection with a non-WGS 84 CRS context; based on https://geojson.org/schema/FeatureCollection.json. Attention: the coordinate order is fixed: Longitude/Easting/Westing/X first, followed by Latitude/Northing/Southing/Y, optionally height as third coordinate.", + "title": "AbstractAnyCrsFeatureCollection", + "type": "object", + "required": [ + "type", + "persistableReferenceCrs", + "features" + ], + "properties": { + "CoordinateReferenceSystemID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-CoordinateReferenceSystem:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "The CRS reference into the CoordinateReferenceSystem catalog.", + "x-osdu-relationship": [ + { + "EntityType": "CoordinateReferenceSystem", + "GroupType": "reference-data" + } + ], + "title": "Coordinate Reference System ID", + "type": "string", + "example": "namespace:reference-data--CoordinateReferenceSystem:BoundCRS.SLB.32021.15851:" + }, + "persistableReferenceCrs": { + "description": "The CRS reference as persistableReference string. If populated, the CoordinateReferenceSystemID takes precedence.", + "type": "string", + "title": "CRS Reference", + "example": "{\"lateBoundCRS\":{\"wkt\":\"PROJCS[\\\"NAD_1927_StatePlane_North_Dakota_South_FIPS_3302\\\",GEOGCS[\\\"GCS_North_American_1927\\\",DATUM[\\\"D_North_American_1927\\\",SPHEROID[\\\"Clarke_1866\\\",6378206.4,294.9786982]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Lambert_Conformal_Conic\\\"],PARAMETER[\\\"False_Easting\\\",2000000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",-100.5],PARAMETER[\\\"Standard_Parallel_1\\\",46.1833333333333],PARAMETER[\\\"Standard_Parallel_2\\\",47.4833333333333],PARAMETER[\\\"Latitude_Of_Origin\\\",45.6666666666667],UNIT[\\\"Foot_US\\\",0.304800609601219],AUTHORITY[\\\"EPSG\\\",32021]]\",\"ver\":\"PE_10_3_1\",\"name\":\"NAD_1927_StatePlane_North_Dakota_South_FIPS_3302\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"32021\"},\"type\":\"LBC\"},\"singleCT\":{\"wkt\":\"GEOGTRAN[\\\"NAD_1927_To_WGS_1984_79_CONUS\\\",GEOGCS[\\\"GCS_North_American_1927\\\",DATUM[\\\"D_North_American_1927\\\",SPHEROID[\\\"Clarke_1866\\\",6378206.4,294.9786982]],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[\\\"NADCON\\\"],PARAMETER[\\\"Dataset_conus\\\",0.0],AUTHORITY[\\\"EPSG\\\",15851]]\",\"ver\":\"PE_10_3_1\",\"name\":\"NAD_1927_To_WGS_1984_79_CONUS\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"15851\"},\"type\":\"ST\"},\"ver\":\"PE_10_3_1\",\"name\":\"NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]\",\"authCode\":{\"auth\":\"SLB\",\"code\":\"32021079\"},\"type\":\"EBC\"}" + }, + "features": { + "type": "array", + "items": { + "title": "AnyCrsGeoJSON Feature", + "type": "object", + "required": [ + "type", + "properties", + "geometry" + ], + "properties": { + "geometry": { + "oneOf": [ + { + "type": "null" + }, + { + "title": "AnyCrsGeoJSON Point", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsPoint" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON LineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsLineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON Polygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsPolygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON MultiPoint", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsMultiPoint" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON MultiLineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsMultiLineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON MultiPolygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsMultiPolygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON GeometryCollection", + "type": "object", + "required": [ + "type", + "geometries" + ], + "properties": { + "type": { + "type": "string", + "enum": [ + "AnyCrsGeometryCollection" + ] + }, + "geometries": { + "type": "array", + "items": { + "oneOf": [ + { + "title": "AnyCrsGeoJSON Point", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsPoint" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON LineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsLineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON Polygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsPolygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON MultiPoint", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsMultiPoint" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON MultiLineString", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsMultiLineString" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + }, + { + "title": "AnyCrsGeoJSON MultiPolygon", + "type": "object", + "required": [ + "type", + "coordinates" + ], + "properties": { + "coordinates": { + "type": "array", + "items": { + "type": "array", + "items": { + "minItems": 4, + "type": "array", + "items": { + "minItems": 2, + "type": "array", + "items": { + "type": "number" + } + } + } + } + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsMultiPolygon" + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + } + ] + } + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + } + ] + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsFeature" + ] + }, + "properties": { + "oneOf": [ + { + "type": "null" + }, + { + "type": "object" + } + ] + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + } + } + } + }, + "persistableReferenceUnitZ": { + "description": "The unit of measure for the Z-axis (only for 3-dimensional coordinates, where the CRS does not describe the vertical unit). Note that the direction is upwards positive, i.e. Z means height.", + "type": "string", + "title": "Z-Unit Reference", + "example": "{\"scaleOffset\":{\"scale\":1.0,\"offset\":0.0},\"symbol\":\"m\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}" + }, + "bbox": { + "minItems": 4, + "type": "array", + "items": { + "type": "number" + } + }, + "persistableReferenceVerticalCrs": { + "description": "The VerticalCRS reference as persistableReference string. If populated, the VerticalCoordinateReferenceSystemID takes precedence. The property is null or empty for 2D geometries. For 3D geometries and absent or null persistableReferenceVerticalCrs the vertical CRS is either provided via persistableReferenceCrs's CompoundCRS or it is implicitly defined as EPSG:5714 MSL height.", + "type": "string", + "title": "Vertical CRS Reference", + "example": "{\"authCode\":{\"auth\":\"EPSG\",\"code\":\"5773\"},\"type\":\"LBC\",\"ver\":\"PE_10_3_1\",\"name\":\"EGM96_Geoid\",\"wkt\":\"VERTCS[\\\"EGM96_Geoid\\\",VDATUM[\\\"EGM96_Geoid\\\"],PARAMETER[\\\"Vertical_Shift\\\",0.0],PARAMETER[\\\"Direction\\\",1.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",5773]]\"}" + }, + "type": { + "type": "string", + "enum": [ + "AnyCrsFeatureCollection" + ] + }, + "VerticalCoordinateReferenceSystemID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-CoordinateReferenceSystem:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "The explicit VerticalCRS reference into the CoordinateReferenceSystem catalog. This property stays empty for 2D geometries. Absent or empty values for 3D geometries mean the context may be provided by a CompoundCRS in 'CoordinateReferenceSystemID' or implicitly EPSG:5714 MSL height", + "x-osdu-relationship": [ + { + "EntityType": "CoordinateReferenceSystem", + "GroupType": "reference-data" + } + ], + "title": "Vertical Coordinate Reference System ID", + "type": "string", + "example": "namespace:reference-data--CoordinateReferenceSystem:VerticalCRS.EPSG.5773:" + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAnyCrsFeatureCollection.1.0.0.json" + }, + "osdu:wks:AbstractSpatialLocation:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractSpatialLocation:1.0.0", + "description": "A geographic object which can be described by a set of points.", + "title": "AbstractSpatialLocation", + "type": "object", + "properties": { + "AsIngestedCoordinates": { + "description": "The original or 'as ingested' coordinates (Point, MultiPoint, LineString, MultiLineString, Polygon or MultiPolygon). The name 'AsIngestedCoordinates' was chosen to contrast it to 'OriginalCoordinates', which carries the uncertainty whether any coordinate operations took place before ingestion. In cases where the original CRS is different from the as-ingested CRS, the OperationsApplied can also contain the list of operations applied to the coordinate prior to ingestion. The data structure is similar to GeoJSON FeatureCollection, however in a CRS context explicitly defined within the AbstractAnyCrsFeatureCollection. The coordinate sequence follows GeoJSON standard, i.e. 'eastward/longitude', 'northward/latitude' {, 'upward/height' unless overridden by an explicit direction in the AsIngestedCoordinates.VerticalCoordinateReferenceSystemID}.", + "x-osdu-frame-of-reference": "CRS:", + "title": "As Ingested Coordinates", + "$ref": "#/definitions/osdu:wks:AbstractAnyCrsFeatureCollection:1.0.0" + }, + "SpatialParameterTypeID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-SpatialParameterType:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "A type of spatial representation of an object, often general (e.g. an Outline, which could be applied to Field, Reservoir, Facility, etc.) or sometimes specific (e.g. Onshore Outline, State Offshore Outline, Federal Offshore Outline, 3 spatial representations that may be used by Countries).", + "x-osdu-relationship": [ + { + "EntityType": "SpatialParameterType", + "GroupType": "reference-data" + } + ], + "type": "string" + }, + "QuantitativeAccuracyBandID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-QuantitativeAccuracyBand:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "An approximate quantitative assessment of the quality of a location (accurate to > 500 m (i.e. not very accurate)), to < 1 m, etc.", + "x-osdu-relationship": [ + { + "EntityType": "QuantitativeAccuracyBand", + "GroupType": "reference-data" + } + ], + "type": "string" + }, + "CoordinateQualityCheckRemarks": { + "type": "array", + "description": "Freetext remarks on Quality Check.", + "items": { + "type": "string" + } + }, + "AppliedOperations": { + "description": "The audit trail of operations applied to the coordinates from the original state to the current state. The list may contain operations applied prior to ingestion as well as the operations applied to produce the Wgs84Coordinates. The text elements refer to ESRI style CRS and Transformation names, which may have to be translated to EPSG standard names.", + "title": "Operations Applied", + "type": "array", + "items": { + "type": "string" + }, + "example": [ + "conversion from ED_1950_UTM_Zone_31N to GCS_European_1950; 1 points converted", + "transformation GCS_European_1950 to GCS_WGS_1984 using ED_1950_To_WGS_1984_24; 1 points successfully transformed" + ] + }, + "QualitativeSpatialAccuracyTypeID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-QualitativeSpatialAccuracyType:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "A qualitative description of the quality of a spatial location, e.g. unverifiable, not verified, basic validation.", + "x-osdu-relationship": [ + { + "EntityType": "QualitativeSpatialAccuracyType", + "GroupType": "reference-data" + } + ], + "type": "string" + }, + "CoordinateQualityCheckPerformedBy": { + "type": "string", + "description": "The user who performed the Quality Check." + }, + "SpatialLocationCoordinatesDate": { + "format": "date-time", + "description": "Date when coordinates were measured or retrieved.", + "x-osdu-frame-of-reference": "DateTime", + "type": "string" + }, + "CoordinateQualityCheckDateTime": { + "format": "date-time", + "description": "The date of the Quality Check.", + "x-osdu-frame-of-reference": "DateTime", + "type": "string" + }, + "Wgs84Coordinates": { + "title": "WGS 84 Coordinates", + "description": "The normalized coordinates (Point, MultiPoint, LineString, MultiLineString, Polygon or MultiPolygon) based on WGS 84 (EPSG:4326 for 2-dimensional coordinates, EPSG:4326 + EPSG:5714 (MSL) for 3-dimensional coordinates). This derived coordinate representation is intended for global discoverability only. The schema of this substructure is identical to the GeoJSON FeatureCollection https://geojson.org/schema/FeatureCollection.json. The coordinate sequence follows GeoJSON standard, i.e. longitude, latitude {, height}", + "$ref": "#/definitions/osdu:wks:AbstractFeatureCollection:1.0.0" + }, + "SpatialGeometryTypeID": { + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-SpatialGeometryType:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "Indicates the expected look of the SpatialParameterType, e.g. Point, MultiPoint, LineString, MultiLineString, Polygon, MultiPolygon. The value constrains the type of geometries in the GeoJSON Wgs84Coordinates and AsIngestedCoordinates.", + "x-osdu-relationship": [ + { + "EntityType": "SpatialGeometryType", + "GroupType": "reference-data" + } + ], + "type": "string" + } + }, + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractSpatialLocation.1.0.0.json" + }, + "osdu:wks:dataset--GenericDataset:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:dataset--GenericDataset:1.0.0", + "description": "An auto-generated placeholder schema representing dataset group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.", + "title": "GenericDataset", + "type": "object", + "properties": { + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0" + }, + "data": { + "allOf": [ + { + "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0" + }, + { + "description": "Schema fragment holding properties common for all datasets.", + "title": "AbstractDataset", + "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" + ] + }, + "Description": { + "description": "An optional, textual description of the dataset.", + "type": "string", + "title": "Description", + "example": "As originally delivered by ACME.com." + }, + "DatasetProperties": { + "description": "Placeholder for a specialization.", + "type": "object", + "title": "Dataset Properties", + "example": {} + }, + "TotalSize": { + "format": "integer", + "pattern": "^[0-9]+$", + "description": "Total size of the dataset in bytes; for files it is the same as declared in FileSourceInfo.FileSize or the sum of all individual files. Implemented as string. The value must be convertible to a long integer (sizes can become very large).", + "type": "string", + "title": "Total Size", + "example": 13245217273 + }, + "EncodingFormatTypeID": { + "x-osdu-relationship": [ + { + "EntityType": "EncodingFormatType", + "GroupType": "reference-data" + } + ], + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-EncodingFormatType:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "EncodingFormatType ID reference value relationship. It can me a mime-type or media-type.", + "type": "string", + "title": "Encoding Format Type ID", + "example": "namespace:reference-data--EncodingFormatType:text%2Fcsv:" + }, + "Name": { + "description": "An optional name of the dataset, e.g. a user friendly file or file collection name.", + "type": "string", + "title": "Name", + "example": "Dataset X221/15" + }, + "SchemaFormatTypeID": { + "x-osdu-relationship": [ + { + "EntityType": "SchemaFormatType", + "GroupType": "reference-data" + } + ], + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-SchemaFormatType:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "Relationship to the SchemaFormatType reference value.", + "type": "string", + "title": "Schema Format Type ID", + "example": "namespace:reference-data--SchemaFormatType:CWLS%20LAS3:" + } + } + } + ] + }, + "kind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[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": "osdu:wks:dataset--GenericDataset:1.0.0" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1562066009929332 + }, + "tags": { + "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.", + "additionalProperties": { + "type": "string" + }, + "title": "Tag Dictionary", + "type": "object", + "example": { + "NameOfKey": "String value" + } + }, + "modifyUser": { + "description": "The user reference, which created this version of this resource object. Set by the System.", + "title": "Resource Object Version Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "modifyTime": { + "format": "date-time", + "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Version Creation DateTime", + "type": "string", + "example": "2020-12-16T11:52:24.477Z" + }, + "createTime": { + "format": "date-time", + "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Creation DateTime", + "type": "string", + "example": "2020-12-16T11:46:20.163Z" + }, + "meta": { + "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0" + } + }, + "legal": { + "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.", + "title": "Legal Tags", + "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0" + }, + "createUser": { + "description": "The user reference, which created the first version of this resource object. Set by the System.", + "title": "Resource Object Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "id": { + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:dataset\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$", + "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "namespace:dataset--GenericDataset:b792625f-5cb1-56c5-9699-eb35259e1f9f" + } + }, + "required": [ + "kind", + "acl", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/dataset/GenericDataset.1.0.0.json" + }, + "osdu:wks:AbstractLegalTags:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractLegalTags:1.0.0", + "description": "Legal meta data like legal tags, relevant other countries, legal status. This structure is included by the SystemProperties \"legal\", which is part of all OSDU records. Not extensible.", + "additionalProperties": false, + "title": "Legal Meta Data", + "type": "object", + "properties": { + "legaltags": { + "description": "The list of legal tags, which resolve to legal properties (like country of origin, export classification code, etc.) and rules with the help of the Compliance Service.", + "title": "Legal Tags", + "type": "array", + "items": { + "type": "string" + } + }, + "otherRelevantDataCountries": { + "description": "The list of other relevant data countries as an array of two-letter country codes, see https://en.wikipedia.org/wiki/ISO_3166-1_alpha-2.", + "title": "Other Relevant Data Countries", + "type": "array", + "items": { + "pattern": "^[A-Z]{2}$", + "type": "string" + } + }, + "status": { + "pattern": "^(compliant|uncompliant)$", + "description": "The legal status. Set by the system after evaluation against the compliance rules associated with the \"legaltags\" using the Compliance Service.", + "title": "Legal Status", + "type": "string" + } + }, + "required": [ + "legaltags", + "otherRelevantDataCountries" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalTags.1.0.0.json" + }, + "osdu:wks:master-data--GenericMasterData:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:master-data--GenericMasterData:1.0.0", + "description": "An auto-generated placeholder schema representing master-data group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.", + "title": "GenericMasterData", + "type": "object", + "properties": { + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0" + }, + "data": { + "allOf": [ + { + "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0" + } + ] + }, + "kind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[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": "osdu:wks:master-data--GenericMasterData:1.0.0" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1562066009929332 + }, + "tags": { + "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.", + "additionalProperties": { + "type": "string" + }, + "title": "Tag Dictionary", + "type": "object", + "example": { + "NameOfKey": "String value" + } + }, + "modifyUser": { + "description": "The user reference, which created this version of this resource object. Set by the System.", + "title": "Resource Object Version Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "modifyTime": { + "format": "date-time", + "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Version Creation DateTime", + "type": "string", + "example": "2020-12-16T11:52:24.477Z" + }, + "createTime": { + "format": "date-time", + "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Creation DateTime", + "type": "string", + "example": "2020-12-16T11:46:20.163Z" + }, + "meta": { + "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0" + } + }, + "legal": { + "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.", + "title": "Legal Tags", + "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0" + }, + "createUser": { + "description": "The user reference, which created the first version of this resource object. Set by the System.", + "title": "Resource Object Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "id": { + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:master-data\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$", + "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "namespace:master-data--GenericMasterData:9ca8054c-bce6-5a3a-b51d-f216fb1085a5" + } + }, + "required": [ + "id", + "kind", + "acl", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/master-data/GenericMasterData.1.0.0.json" + }, + "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0", + "description": "An auto-generated placeholder schema representing work-product-component group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.", + "title": "GenericWorkProductComponent", + "type": "object", + "properties": { + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0" + }, + "data": { + "allOf": [ + { + "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0" + }, + { + "description": "Generic reference object containing the universal group-type properties of a Work Product Component for inclusion in data type specific Work Product Component objects", + "title": "AbstractWPCGroupType", + "type": "object", + "properties": { + "Datasets": { + "type": "array", + "items": { + "x-osdu-relationship": [ + { + "GroupType": "dataset" + } + ], + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:dataset\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*)$", + "description": "The SRN which identifies this OSDU File resource.", + "type": "string" + } + }, + "IsDiscoverable": { + "description": "A flag that indicates if the work product component is searchable, which means covered in the search index.", + "type": "boolean" + }, + "Artefacts": { + "description": "An array of Artefacts - each artefact has a Role, Resource tuple. An artefact is distinct from the file, in the sense certain valuable information is generated during loading process (Artefact generation process). Examples include retrieving location data, performing an OCR which may result in the generation of artefacts which need to be preserved distinctly", + "type": "array", + "items": { + "type": "object", + "properties": { + "ResourceID": { + "x-osdu-relationship": [ + { + "GroupType": "dataset" + } + ], + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:dataset\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*)$", + "description": "The SRN which identifies this OSDU Artefact resource.", + "type": "string" + }, + "ResourceKind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$", + "description": "The kind or schema ID of the artefact. Resolvable with the Schema Service.", + "type": "string" + }, + "RoleID": { + "x-osdu-relationship": [ + { + "EntityType": "ArtefactRole", + "GroupType": "reference-data" + } + ], + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ArtefactRole:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "The SRN of this artefact's role.", + "type": "string" + } + } + } + }, + "IsExtendedLoad": { + "description": "A flag that indicates if the work product component is undergoing an extended load. It reflects the fact that the work product component is in an early stage and may be updated before finalization.", + "type": "boolean" + } + } + } + ] + }, + "kind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[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": "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1562066009929332 + }, + "tags": { + "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.", + "additionalProperties": { + "type": "string" + }, + "title": "Tag Dictionary", + "type": "object", + "example": { + "NameOfKey": "String value" + } + }, + "modifyUser": { + "description": "The user reference, which created this version of this resource object. Set by the System.", + "title": "Resource Object Version Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "modifyTime": { + "format": "date-time", + "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Version Creation DateTime", + "type": "string", + "example": "2020-12-16T11:52:24.477Z" + }, + "createTime": { + "format": "date-time", + "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Creation DateTime", + "type": "string", + "example": "2020-12-16T11:46:20.163Z" + }, + "meta": { + "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0" + } + }, + "legal": { + "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.", + "title": "Legal Tags", + "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0" + }, + "createUser": { + "description": "The user reference, which created the first version of this resource object. Set by the System.", + "title": "Resource Object Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "id": { + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:work-product-component\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$", + "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "namespace:work-product-component--GenericWorkProductComponent:80575f9c-fc7b-516b-b44f-996874b9f775" + } + }, + "required": [ + "kind", + "acl", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/work-product-component/GenericWorkProductComponent.1.0.0.json" + }, + "osdu:wks:work-product--GenericWorkProduct:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:work-product--GenericWorkProduct:1.0.0", + "description": "An auto-generated placeholder schema representing work-product group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.", + "title": "GenericWorkProduct", + "type": "object", + "properties": { + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry", + "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0" + }, + "data": { + "allOf": [ + { + "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0" + }, + { + "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.", + "title": "WorkProduct", + "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": { + "x-osdu-relationship": [ + { + "GroupType": "work-product-component" + } + ], + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:work-product-component\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[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": "#/definitions/osdu:wks: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": "#/definitions/osdu:wks: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": { + "x-osdu-relationship": [], + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*$", + "description": "The object reference identifying the DIRECT, INDIRECT, REFERENCE dependency.", + "type": "string" + }, + "LineageRelationshipType": { + "x-osdu-relationship": [ + { + "EntityType": "LineageRelationshipType", + "GroupType": "reference-data" + } + ], + "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-LineageRelationshipType:[\\w\\-\\.\\:\\%]+:[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" + } + } + } + } + ] + }, + "kind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[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": "osdu:wks:work-product--GenericWorkProduct:1.0.0" + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List", + "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0" + }, + "version": { + "format": "int64", + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "example": 1562066009929332 + }, + "tags": { + "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.", + "additionalProperties": { + "type": "string" + }, + "title": "Tag Dictionary", + "type": "object", + "example": { + "NameOfKey": "String value" + } + }, + "modifyUser": { + "description": "The user reference, which created this version of this resource object. Set by the System.", + "title": "Resource Object Version Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "modifyTime": { + "format": "date-time", + "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Version Creation DateTime", + "type": "string", + "example": "2020-12-16T11:52:24.477Z" + }, + "createTime": { + "format": "date-time", + "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.", + "title": "Resource Object Creation DateTime", + "type": "string", + "example": "2020-12-16T11:46:20.163Z" + }, + "meta": { + "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.", + "title": "Frame of Reference Meta Data", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0" + } + }, + "legal": { + "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.", + "title": "Legal Tags", + "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0" + }, + "createUser": { + "description": "The user reference, which created the first version of this resource object. Set by the System.", + "title": "Resource Object Creation User Reference", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "id": { + "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:work-product\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$", + "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.", + "title": "Entity ID", + "type": "string", + "example": "namespace:work-product--GenericWorkProduct:955930ee-e6bd-5ae0-a6ee-67ba902e1635" + } + }, + "required": [ + "kind", + "acl", + "legal" + ], + "$id": "https://schema.osdu.opengroup.org/json/work-product/GenericWorkProduct.1.0.0.json" + }, + "osdu:wks:AbstractAccessControlList:1.0.0": { + "x-osdu-inheriting-from-kind": [], + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:AbstractAccessControlList:1.0.0", + "description": "The access control tags associated with this entity. This structure is included by the SystemProperties \"acl\", which is part of all OSDU records. Not extensible.", + "additionalProperties": false, + "title": "Access Control List", + "type": "object", + "properties": { + "viewers": { + "description": "The list of viewers to which this data record is accessible/visible/discoverable formatted as an email (core.common.model.storage.validation.ValidationDoc.EMAIL_REGEX).", + "title": "List of Viewers", + "type": "array", + "items": { + "pattern": "^[a-zA-Z0-9_+&*-]+(?:\\.[a-zA-Z0-9_+&*-]+)*@(?:[a-zA-Z0-9-]+\\.)+[a-zA-Z]{2,7}$", + "type": "string" + } + }, + "owners": { + "description": "The list of owners of this data record formatted as an email (core.common.model.storage.validation.ValidationDoc.EMAIL_REGEX).", + "title": "List of Owners", + "type": "array", + "items": { + "pattern": "^[a-zA-Z0-9_+&*-]+(?:\\.[a-zA-Z0-9_+&*-]+)*@(?:[a-zA-Z0-9-]+\\.)+[a-zA-Z]{2,7}$", + "type": "string" + } + } + }, + "required": [ + "owners", + "viewers" + ], + "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAccessControlList.1.0.0.json" + } + }, + "properties": { + "ReferenceData": { + "description": "Reference-data are submitted as an array of records.", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:reference-data--GenericReferenceData:1.0.0" + } + }, + "MasterData": { + "description": "Master-data are submitted as an array of records.", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:master-data--GenericMasterData:1.0.0" + } + }, + "kind": { + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$", + "description": "The schema identification for the manifest record following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.", + "title": "Manifest Kind", + "type": "string", + "example": "osdu:wks:Manifest:1.0.0" + }, + "Data": { + "description": "Manifest schema for work-product, work-product-component, dataset ensembles. The items in 'Datasets' are processed first since they are referenced by 'WorkProductComponents' ('data.DatasetIDs[]' and 'data.Artefacts[].ResourceID'). The WorkProduct is processed last collecting the WorkProductComponents.", + "type": "object", + "properties": { + "WorkProduct": { + "description": "The work-product component capturing the work-product-component records belonging to this loading/ingestion transaction.", + "$ref": "#/definitions/osdu:wks:work-product--GenericWorkProduct:1.0.0" + }, + "Datasets": { + "description": "The list of 'Datasets' or data containers holding the actual data. The record ids are usually internal surrogate keys enabling the association of dataset records with work-product-component records, namely via 'DatasetIDs' and 'Artefacts.ResourceID' (both referring to 'dataset' group-type entity types).", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:dataset--GenericDataset:1.0.0" + } + }, + "WorkProductComponents": { + "description": "The list of work-product-components records. The record ids are internal surrogate keys enabling the association of work-product-component records with the work-product records.", + "type": "array", + "items": { + "$ref": "#/definitions/osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0" + } + } + } + } + }, + "$id": "https://schema.osdu.opengroup.org/json/manifest/Manifest.1.0.0.json" +} diff --git a/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json new file mode 100644 index 0000000000000000000000000000000000000000..b88045eb90fc2b6f5813f70ce508535592ab4cbc --- /dev/null +++ b/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json @@ -0,0 +1,1177 @@ +{ + "execution_context": { + "Payload": { + "authorization": "Bearer test", + "data-partition-id": "opendes", + "AppKey": "", + "kind_version": "3.0.0" + }, + "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", + "$filename": "load_Wellbore.1.0.0_350112350400.json", + "manifest": [ + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + }, + { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { + "id": "opendes:master-data/Wellbore:350112350400", + "kind": "opendes:osdu:TestMaster:0.3.0", + "groupType": "master-data", + "version": 1, + "acl": { + "owners": [ + "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ], + "viewers": [ + "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com" + ] + }, + "legal": { + "legaltags": [ + "opendes-demo-legaltag" + ], + "otherRelevantDataCountries": [ + "srn:opendes:master-data/GeoPoliticalEntity:USA:" + ], + "status": "srn:opendes:reference-data/LegalStatus:public:1111" + }, + "resourceHostRegionIDs": [ + "srn:opendes:reference-data/OSDURegion:US-EAST:" + ], + "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00", + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:", + "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", + "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", + "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", + "data": { + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" + } + ], + "Data": {} + } + ] + }, + "workflow_name": "osdu_ingest", + "run_id": "foo" +} diff --git a/tests/plugin-unit-tests/data/master/r3_Well.json b/tests/plugin-unit-tests/data/master/r3_Well.json new file mode 100644 index 0000000000000000000000000000000000000000..f65e5603835f8bb6dce12fc2c04da4547949ce2a --- /dev/null +++ b/tests/plugin-unit-tests/data/master/r3_Well.json @@ -0,0 +1,72 @@ +{ + "MasterData": [ + { + "id": "osdu:master-data--Well:BA050018", + "kind": "osdu:wks:master-data--Well:1.0.0", + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:Public:", + "Source": "NL_TNO", + "SpatialLocation": { + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "geometry": { + "type": "Point", + "coordinates": [ + 3.51906683, + 55.68101428 + ] + }, + "properties": {} + } + ] + } + }, + "FacilityID": "10110909", + "FacilityTypeID": "osdu:reference-data--FacilityType:WELL:", + "FacilityOperator": [ + { + "FacilityOperatorID": "410464", + "FacilityOperatorOrganisationID": "osdu:master-data--Organisation:HESS:" + } + ], + "FacilityName": "A05-01", + "FacilityNameAlias": [ + { + "AliasName": "A05-01", + "AliasNameTypeID": "osdu:reference-data--AliasNameType:WELL_NAME:" + } + ], + "FacilityEvent": [ + { + "FacilityEventTypeID": "osdu:reference-data--FacilityEventType:SPUD_DATE:", + "EffectiveDateTime": "1999-06-03T00:00:00" + } + ], + "VerticalMeasurements": [ + { + "VerticalMeasurementID": "Kelly Bushing", + "VerticalMeasurement": 36.6, + "VerticalMeasurementPathID": "osdu:reference-data--VerticalMeasurementPath:DEPTH_DATUM_ELEV:" + } + ], + "NameAliases": [], + "GeoContexts": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + } + } + ] +} diff --git a/tests/plugin-unit-tests/data/master/ref_result_r3_Well.json b/tests/plugin-unit-tests/data/master/ref_result_r3_Well.json new file mode 100644 index 0000000000000000000000000000000000000000..80a7d8d7b92b8915c76c7bf9fd41fcb6d9a340e0 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/ref_result_r3_Well.json @@ -0,0 +1,8 @@ +[ + "osdu:reference-data--ResourceSecurityClassification:Public:", + "osdu:reference-data--FacilityType:WELL:", + "osdu:master-data--Organisation:HESS:", + "osdu:reference-data--AliasNameType:WELL_NAME:", + "osdu:reference-data--FacilityEventType:SPUD_DATE:", + "osdu:reference-data--VerticalMeasurementPath:DEPTH_DATUM_ELEV:" +] diff --git a/tests/plugin-unit-tests/data/master/traversal_r3_Well.json b/tests/plugin-unit-tests/data/master/traversal_r3_Well.json new file mode 100644 index 0000000000000000000000000000000000000000..26db0a3849392f65f99518f5122de68913bf1f95 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/traversal_r3_Well.json @@ -0,0 +1,74 @@ +[ + { + "entity": { + "id": "osdu:master-data--Well:BA050018", + "kind": "osdu:wks:master-data--Well:1.0.0", + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:Public:", + "Source": "NL_TNO", + "SpatialLocation": { + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "geometry": { + "type": "Point", + "coordinates": [ + 3.51906683, + 55.68101428 + ] + }, + "properties": {} + } + ] + } + }, + "FacilityID": "10110909", + "FacilityTypeID": "osdu:reference-data--FacilityType:WELL:", + "FacilityOperator": [ + { + "FacilityOperatorID": "410464", + "FacilityOperatorOrganisationID": "osdu:master-data--Organisation:HESS:" + } + ], + "FacilityName": "A05-01", + "FacilityNameAlias": [ + { + "AliasName": "A05-01", + "AliasNameTypeID": "osdu:reference-data--AliasNameType:WELL_NAME:" + } + ], + "FacilityEvent": [ + { + "FacilityEventTypeID": "osdu:reference-data--FacilityEventType:SPUD_DATE:", + "EffectiveDateTime": "1999-06-03T00:00:00" + } + ], + "VerticalMeasurements": [ + { + "VerticalMeasurementID": "Kelly Bushing", + "VerticalMeasurement": 36.6, + "VerticalMeasurementPathID": "osdu:reference-data--VerticalMeasurementPath:DEPTH_DATUM_ELEV:" + } + ], + "NameAliases": [], + "GeoContexts": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + } + }, + "schema": "GenericMasterData.1.0.0.json", + "parent": + } +] diff --git a/tests/plugin-unit-tests/data/other/ExtractedIds.json b/tests/plugin-unit-tests/data/other/ExtractedIds.json new file mode 100644 index 0000000000000000000000000000000000000000..bc0cd34b713e47b27b5a325825088aa00648bebe --- /dev/null +++ b/tests/plugin-unit-tests/data/other/ExtractedIds.json @@ -0,0 +1,4 @@ +[ + "some_test_id", + "some_test_id:12345" +] diff --git a/tests/plugin-unit-tests/data/other/SearchResponseValid.json b/tests/plugin-unit-tests/data/other/SearchResponseValid.json index 6e8fe92db01ccb71b541a6f204b45499414ec80f..a8303d36fec5cbea37b27a7e089acb82b3563b25 100644 --- a/tests/plugin-unit-tests/data/other/SearchResponseValid.json +++ b/tests/plugin-unit-tests/data/other/SearchResponseValid.json @@ -1,5 +1,9 @@ { "results": [ + { + "id": "some_test_id", + "version": 12345 + } ], "aggregations": null, "totalCount": 45 diff --git a/tests/plugin-unit-tests/data/batchManifest/Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/surrogate/Wellbore.0.3.0.json similarity index 100% rename from tests/plugin-unit-tests/data/batchManifest/Wellbore.0.3.0.json rename to tests/plugin-unit-tests/data/surrogate/Wellbore.0.3.0.json diff --git a/tests/plugin-unit-tests/data/workProduct/r3_Welllog.json b/tests/plugin-unit-tests/data/workProduct/r3_Welllog.json new file mode 100644 index 0000000000000000000000000000000000000000..e32fb1e59373b22ba6a0293fa6d6955d7ed06779 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/r3_Welllog.json @@ -0,0 +1,141 @@ +{ + "kind": "osdu:wks:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [], + "Data": { + "WorkProduct": { + "kind": "osdu:wks:work-product--WorkProduct:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "Name": "AKM-11 LOG", + "Description": "Well Log", + "Components": [ + "surrogate-key:wpc-1" + ] + } + }, + "WorkProductComponents": [ + { + "id": "surrogate-key:wpc-1", + "kind": "osdu:wks:work-product-component--WellLog:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "Name": "AKM-11 LOG", + "Description": "Well Log", + "Datasets": [ + "surrogate-key:file-1" + ], + "WellboreID": "osdu:master-data--Wellbore:1013:", + "TopMeasuredDepth": 2182.0004, + "BottomMeasuredDepth": 2481.0, + "Curves": [ + { + "Mnemonic": "DEPT", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:M:" + }, + { + "Mnemonic": "GR", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:GAPI:" + }, + { + "Mnemonic": "DT", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:US/F:" + }, + { + "Mnemonic": "RHOB", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:" + }, + { + "Mnemonic": "DRHO", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:" + }, + { + "Mnemonic": "NPHI", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:V/V:" + } + ] + } + } + ], + "Datasets": [ + { + "id": "surrogate-key:file-1", + "kind": "osdu:wks:dataset--File.Generic:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:LAS 2:", + "DatasetProperties": { + "FileSourceInfo": { + "FileSource": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las", + "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las" + } + } + } + }, + { + "id": "surrogate-key:file-2", + "kind": "osdu:wks:dataset--File.Generic:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:LAS 2:", + "DatasetProperties": { + "FileSourceInfo": { + "FileSource": "", + "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las" + } + } + } + } + ] + } +} diff --git a/tests/plugin-unit-tests/data/workProduct/ref_result_r3_Welllog.json b/tests/plugin-unit-tests/data/workProduct/ref_result_r3_Welllog.json new file mode 100644 index 0000000000000000000000000000000000000000..190471fcc48114edab869d2a8696d0e97aa1d907 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/ref_result_r3_Welllog.json @@ -0,0 +1,9 @@ +[ + "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "osdu:master-data--Wellbore:1013:", + "osdu:reference-data--UnitOfMeasure:M:", + "osdu:reference-data--UnitOfMeasure:GAPI:", + "osdu:reference-data--UnitOfMeasure:US/F:", + "osdu:reference-data--UnitOfMeasure:G/C3:", + "osdu:reference-data--UnitOfMeasure:V/V:" +] diff --git a/tests/plugin-unit-tests/data/workProduct/traversal_r3_Welllog.json b/tests/plugin-unit-tests/data/workProduct/traversal_r3_Welllog.json new file mode 100644 index 0000000000000000000000000000000000000000..55eba5b875dfe0bd25066020137177c8fa7afa74 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/traversal_r3_Welllog.json @@ -0,0 +1,120 @@ +[ + { + "entity": { + "kind": "osdu:wks:work-product--WorkProduct:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "Name": "AKM-11 LOG", + "Description": "Well Log", + "Components": [ + "surrogate-key:wpc-1" + ] + } + }, + "schema": "", + "type": "wpc" + }, + { + "entity": { + "id": "surrogate-key:wpc-1", + "kind": "osdu:wks:work-product-component--WellLog:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "Name": "AKM-11 LOG", + "Description": "Well Log", + "Datasets": [ + "surrogate-key:file-1" + ], + "WellboreID": "osdu:master-data--Wellbore:1013:", + "TopMeasuredDepth": 2182.0004, + "BottomMeasuredDepth": 2481.0, + "Curves": [ + { + "Mnemonic": "DEPT", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:M:" + }, + { + "Mnemonic": "GR", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:GAPI:" + }, + { + "Mnemonic": "DT", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:US/F:" + }, + { + "Mnemonic": "RHOB", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:" + }, + { + "Mnemonic": "DRHO", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:" + }, + { + "Mnemonic": "NPHI", + "TopDepth": 2182.0, + "BaseDepth": 2481.0, + "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:", + "CurveUnit": "osdu:reference-data--UnitOfMeasure:V/V:" + } + ] + } + }, + "schema": "" + }, + { + "entity": { + "id": "surrogate-key:file-1", + "kind": "osdu:wks:dataset--File.Generic:1.0.0", + "acl": { + "owners": [], + "viewers": [] + }, + "legal": { + "legaltags": [], + "otherRelevantDataCountries": [] + }, + "data": { + "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:", + "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:LAS 2:", + "DatasetProperties": { + "FileSourceInfo": { + "FileSource": "", + "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las" + } + } + } + }, + "schema": "" + } +] diff --git a/tests/plugin-unit-tests/file_paths.py b/tests/plugin-unit-tests/file_paths.py index f03b7c5a8709fce25d5dd564ca373f067b18a1ee..8630f262aa813390b6bdf015aa3245d6f85c14dc 100644 --- a/tests/plugin-unit-tests/file_paths.py +++ b/tests/plugin-unit-tests/file_paths.py @@ -18,9 +18,11 @@ import os DATA_PATH_PREFIX = f"{os.path.dirname(__file__)}/data" MANIFEST_GENERIC_SCHEMA_PATH = f"{DATA_PATH_PREFIX}/manifests/schema_Manifest.1.0.0.json" +MANIFEST_NEW_GENERIC_SCHEMA_PATH = f"{DATA_PATH_PREFIX}/manifests/new_schema_Manifest.1.0.0.json" MANIFEST_GENERIC_PATH = f"{DATA_PATH_PREFIX}/manifests/Manifest.1.0.0.json" MANIFEST_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/Wellbore.0.3.0.json" +MANIFEST_BATCH_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/batch_Wellbore.0.3.0.json" 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" SCHEMA_GENERIC_MASTERDATA_PATH = f"{DATA_PATH_PREFIX}/master/schema_GenericMasterData.1.0.0.json" @@ -34,11 +36,23 @@ SCHEMA_WORK_PRODUCT_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_WorkPro 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" +SURROGATE_MANIFEST_WELLBORE = f"{DATA_PATH_PREFIX}/surrogate/Wellbore.0.3.0.json" + MANIFEST_EMPTY_PATH = f"{DATA_PATH_PREFIX}/invalid/EmptyManifest.json" TRAVERSAL_MANIFEST_EMPTY_PATH = f"{DATA_PATH_PREFIX}/invalid/TraversalEmptyManifest.json" SEARCH_VALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseValid.json" SEARCH_INVALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseInvalid.json" +SEARCH_EXTRACTED_IDS_PATH = f"{DATA_PATH_PREFIX}/other/ExtractedIds.json" + + +SURROGATE_MANIFEST_WELLBORE = f"{DATA_PATH_PREFIX}/surrogate/Wellbore.0.3.0.json" + +MANIFEST_WELL_PATH = f"{DATA_PATH_PREFIX}/master/r3_Well.json" +TRAVERSAL_WELL_PATH = f"{DATA_PATH_PREFIX}/master/traversal_r3_Well.json" +REF_RESULT_WELL_PATH = f"{DATA_PATH_PREFIX}/master/ref_result_r3_Well.json" -BATCH_MANIFEST_WELLBORE = f"{DATA_PATH_PREFIX}/batchManifest/Wellbore.0.3.0.json" +MANIFEST_WELLLOG_PATH = f"{DATA_PATH_PREFIX}/workProduct/r3_Welllog.json" +TRAVERSAL_WELLLOG_PATH = f"{DATA_PATH_PREFIX}/workProduct/traversal_r3_Welllog.json" +REF_RESULT_WELLLOG_PATH = f"{DATA_PATH_PREFIX}/workProduct/ref_result_r3_Welllog.json" diff --git a/tests/plugin-unit-tests/test_manifest_analyzer.py b/tests/plugin-unit-tests/test_manifest_analyzer.py index 91521e5b5ee983b21b94fdf397c4195649e89cf9..a93a278b044c698542f0a5c309f2416f7bb481fd 100644 --- a/tests/plugin-unit-tests/test_manifest_analyzer.py +++ b/tests/plugin-unit-tests/test_manifest_analyzer.py @@ -22,7 +22,7 @@ import sys from functools import partial import pytest -from file_paths import BATCH_MANIFEST_WELLBORE +from file_paths import SURROGATE_MANIFEST_WELLBORE sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") @@ -71,7 +71,7 @@ class TestManifestAnalyzer(object): @pytest.fixture() def manifest_analyzer(self): - with open(BATCH_MANIFEST_WELLBORE) as f: + with open(SURROGATE_MANIFEST_WELLBORE) as f: data = json.load(f) context = Context(data_partition_id="test", app_key="test") token_refresher = AirflowTokenRefresher() @@ -184,7 +184,7 @@ class TestManifestAnalyzer(object): "If absent parents are in storage, there are no orphaned child then." def test_real_data(self): - with open(BATCH_MANIFEST_WELLBORE) as f: + with open(SURROGATE_MANIFEST_WELLBORE) as f: data = json.load(f) data = [ManifestEntity(entity=e, schema="") for e in data] context = Context(data_partition_id="test", app_key="test") diff --git a/tests/plugin-unit-tests/test_manifest_integrity.py b/tests/plugin-unit-tests/test_manifest_integrity.py new file mode 100644 index 0000000000000000000000000000000000000000..1bae767e337fa950250f74c1d9d4f0113073bf3b --- /dev/null +++ b/tests/plugin-unit-tests/test_manifest_integrity.py @@ -0,0 +1,72 @@ +# Copyright 2021 Google LLC +# Copyright 2021 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import 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") + +import pytest +from mock_providers import get_test_credentials +from file_paths import MANIFEST_WELL_PATH, REF_RESULT_WELL_PATH, MANIFEST_WELLLOG_PATH, REF_RESULT_WELLLOG_PATH +from libs.refresh_token import AirflowTokenRefresher +from libs.manifest_integrity import ManifestIntegrity + + +class TestIntegrityProvider: + + @pytest.fixture + def manifest_integrity(self, monkeypatch, conf_path: str): + with open(conf_path) as f: + conf = json.load(f) + manifest_integrity = ManifestIntegrity("", AirflowTokenRefresher(get_test_credentials()), conf) + return manifest_integrity + + @pytest.mark.parametrize( + "conf_path,ref_result_file", + [ + pytest.param( + MANIFEST_WELL_PATH, + REF_RESULT_WELL_PATH, + id="Validate manifest Master") + ] + ) + def test_extract_references_master(self, monkeypatch, manifest_integrity, conf_path: str, ref_result_file: str): + with open(ref_result_file) as f: + expected_result = json.load(f) + + manifest_integrity._collect_ids(manifest_integrity.context) + manifest_records = manifest_integrity._extract_references( + manifest_integrity.context["MasterData"][0]) + assert set(manifest_records) == set(expected_result) + + @pytest.mark.parametrize( + "conf_path,ref_result_file", + [ + pytest.param( + MANIFEST_WELLLOG_PATH, + REF_RESULT_WELLLOG_PATH, + id="Validate manifest WPC") + ] + ) + def test_extract_references_wpc(self, monkeypatch, manifest_integrity, conf_path: str, ref_result_file: str): + with open(ref_result_file) as f: + expected_result = json.load(f) + manifest_integrity._collect_ids(manifest_integrity.context) + manifest_records = manifest_integrity._extract_references( + manifest_integrity.context["Data"]["WorkProductComponents"][0]) + assert set(manifest_records) == set(expected_result) diff --git a/tests/plugin-unit-tests/test_manifest_traversal.py b/tests/plugin-unit-tests/test_manifest_traversal.py index 54dbbd6e5e0e16b41d805109f8fe5d2859bb06bd..38dcd75e7eef6c0b4d4227d2a2d8670d35d7d819 100644 --- a/tests/plugin-unit-tests/test_manifest_traversal.py +++ b/tests/plugin-unit-tests/test_manifest_traversal.py @@ -36,7 +36,7 @@ class TestManifestTraversal: with open(manifest_schema_file) as f: manifest_schema = json.load(f) traversal = ManifestTraversal( - conf_manifest_file["execution_context"], + conf_manifest_file["execution_context"].get("manifest", []), manifest_schema ) return traversal @@ -62,7 +62,8 @@ class TestManifestTraversal: traversal_manifest = json.load(f) traversal_manifest = [ManifestEntity(**e) for e in traversal_manifest] manifest_records = manifest_traversal.traverse_manifest() - assert manifest_records == traversal_manifest + for m in manifest_records: + assert m in traversal_manifest, f"Expected {traversal_manifest}\nGot {manifest_records}" @pytest.mark.parametrize( "manifest_file,manifest_schema_file,traversal_manifest_file", diff --git a/tests/plugin-unit-tests/test_operators_r3.py b/tests/plugin-unit-tests/test_operators_r3.py index 42b91e8ae777891fbec4a750d829d95e167b989f..7553f24cdab64bc0de703b25ec6945e22383fba5 100644 --- a/tests/plugin-unit-tests/test_operators_r3.py +++ b/tests/plugin-unit-tests/test_operators_r3.py @@ -32,7 +32,7 @@ import requests from file_paths import ( MANIFEST_WELLBORE_VALID_PATH, - SEARCH_VALID_RESPONSE_PATH, MANIFEST_GENERIC_SCHEMA_PATH) + SEARCH_VALID_RESPONSE_PATH, MANIFEST_GENERIC_SCHEMA_PATH, MANIFEST_BATCH_WELLBORE_VALID_PATH) from operators.process_manifest_r3 import ProcessManifestOperatorR3, SchemaValidator, \ ManifestProcessor from operators.search_record_id import SearchRecordIdOperator @@ -56,6 +56,17 @@ class MockStorageResponse(requests.Response): class TestOperators(object): + def _create_batch_task(self, operator: ClassVar[CustomOperator]) -> (CustomOperator, dict): + with open(MANIFEST_BATCH_WELLBORE_VALID_PATH) as f: + conf = json.load(f) + dag = DAG(dag_id='batch_osdu_ingest', start_date=datetime.now()) + task: CustomOperator = operator(dag=dag, task_id='anytask') + ti = TaskInstance(task=task, execution_date=datetime.now()) + + context = ti.get_template_context() + context["dag_run"] = MockDagRun(conf) + return task, context + def _create_task(self, operator: ClassVar[CustomOperator]) -> (CustomOperator, dict): with open(MANIFEST_WELLBORE_VALID_PATH) as f: conf = json.load(f) @@ -75,6 +86,7 @@ class TestOperators(object): return manifest_schema monkeypatch.setattr(SchemaValidator, "get_schema", _get_common_schema) + monkeypatch.setattr(SchemaValidator, "_validate_against_schema", lambda *args, **kwargs: None) monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj, entities: entities) monkeypatch.setattr(ManifestProcessor, "save_record_to_storage", lambda obj, headers, request_data: MockStorageResponse()) @@ -85,6 +97,25 @@ class TestOperators(object): task.pre_execute(context) task.execute(context) + def test_process_manifest_r3_operator_batch(self, monkeypatch): + + def _get_common_schema(*args, **kwargs): + with open(MANIFEST_GENERIC_SCHEMA_PATH) as f: + manifest_schema = json.load(f) + return manifest_schema + + monkeypatch.setattr(SchemaValidator, "get_schema", _get_common_schema) + monkeypatch.setattr(SchemaValidator, "_validate_against_schema", lambda *args, **kwargs: None) + monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj, entities: entities) + monkeypatch.setattr(ManifestProcessor, "save_record_to_storage", + lambda obj, headers, request_data: MockStorageResponse()) + monkeypatch.setattr(FileHandler, "upload_file", + lambda *args, **kwargs: "test") + + task, context = self._create_batch_task(ProcessManifestOperatorR3) + task.pre_execute(context) + task.execute(context) + def _test_update_status_operator(self, monkeypatch, status: UpdateStatusOperator.prev_ti_state): monkeypatch.setattr(UpdateStatusOperator, "get_previous_ti_statuses", lambda obj, context: status) diff --git a/tests/plugin-unit-tests/test_schema_validator_r3.py b/tests/plugin-unit-tests/test_schema_validator_r3.py index a592013d17fb9e9849e8c450a9bb3bcca9603d8a..2e49f9807cd75a80d82c908bfc647a5fe8bdb32d 100644 --- a/tests/plugin-unit-tests/test_schema_validator_r3.py +++ b/tests/plugin-unit-tests/test_schema_validator_r3.py @@ -36,7 +36,7 @@ from file_paths import ( MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, MANIFEST_WELLBORE_VALID_PATH, MANIFEST_GENERIC_PATH, - MANIFEST_GENERIC_SCHEMA_PATH, + MANIFEST_NEW_GENERIC_SCHEMA_PATH, SCHEMA_WELLBORE_VALID_PATH, TRAVERSAL_WELLBORE_VALID_PATH, TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, @@ -117,25 +117,6 @@ class TestSchemaValidator: validated_records = schema_validator.validate_manifest(manifest_file) assert len(manifest_file) == len(validated_records) - @pytest.mark.parametrize( - "manifest_file,traversal_manifest_file,schema_file", - [ - pytest.param( - MANIFEST_EMPTY_PATH, TRAVERSAL_MANIFEST_EMPTY_PATH, None, - id="Empty Manifest"), - ] - ) - def test_schema_validator_empty_manifest(self, - monkeypatch, - schema_validator: SchemaValidator, - manifest_file: str, - traversal_manifest_file: str, - schema_file: str): - with open(traversal_manifest_file) as f: - manifest_file = json.load(f) - with pytest.raises(EmptyManifestError): - schema_validator.validate_manifest(manifest_file) - @pytest.mark.parametrize( "traversal_manifest_file,schema_file", [ @@ -205,7 +186,7 @@ class TestSchemaValidator: [ pytest.param( MANIFEST_GENERIC_PATH, - MANIFEST_GENERIC_SCHEMA_PATH, + MANIFEST_NEW_GENERIC_SCHEMA_PATH, id="Valid generic manifest"), ] ) @@ -224,4 +205,16 @@ class TestSchemaValidator: with open(manifest_file) as f: manifest_content = json.load(f) monkeypatch.setattr(schema_validator, "get_schema", mock_get_schema) - schema_validator.validate_common_schema({"manifest": manifest_content}) + schema_validator.validate_common_schema(manifest_content) + + def test_delete_refs(self): + context = Context(app_key="", data_partition_id="") + validator = SchemaValidator( + "", + AirflowTokenRefresher(get_test_credentials()), + context + ) + manifest = {'manifest': {'ReferenceData': [], 'MasterData': [], 'kind': 'osdu:wks:Manifest:1.0.0', 'Data': {}}} + schema = { "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$id": "https://schema.osdu.opengroup.org/json/manifest/Manifest.1.0.0.json", "$schema": "http://json-schema.org/draft-07/schema#", "x-osdu-schema-source": "osdu:wks:Manifest:1.0.0", "title": "Load Manifest Schema", "description": "Load manifest applicable for all types defined as 'kind', i.e. registered as schemas with the Schema Service. It supports loading of individual 'records' of any group-type or combinations. The load sequence follows a well-defined sequence. The 'ReferenceData' array is processed first (if populated). The 'MasterData' array is processed second (if populated) second. The 'Data' structure is processed last (if populated). Inside the 'Data' property the 'Datasets' array is processed first, followed by the 'WorkProductComponents' array, the 'WorkProduct' is processed last. Any arrays are ordered. should there be interdependencies, the dependent items must be placed behind their relationship targets, e.g. a master-data Well record must placed in the 'MasterData' array before its Wellbores.", "type": "object", "properties": { "kind": { "description": "The schema identification for the manifest record following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.", "title": "Manifest Kind", "type": "string", "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$", "example": "osdu:wks:Manifest:1.0.0" }, "ReferenceData": { "description": "Reference-data are submitted as an array of records.", "type": "array", "items": { "$ref": "GenericReferenceData.1.0.0.json" } }, "MasterData": { "description": "Master-data are submitted as an array of records.", "type": "array", "items": { "$ref": "GenericMasterData.1.0.0.json" } }, "Data": { "description": "Manifest schema for work-product, work-product-component, dataset ensembles. The items in 'Datasets' are processed first since they are referenced by 'WorkProductComponents' ('data.DatasetIDs[]' and 'data.Artefacts[].ResourceID'). The WorkProduct is processed last collecting the WorkProductComponents.", "type": "object", "properties": { "WorkProduct": { "description": "The work-product component capturing the work-product-component records belonging to this loading/ingestion transaction.", "$ref": "GenericWorkProduct.1.0.0.json" }, "WorkProductComponents": { "description": "The list of work-product-components records. The record ids are internal surrogate keys enabling the association of work-product-component records with the work-product records.", "type": "array", "items": { "$ref": "GenericWorkProductComponent.1.0.0.json" } }, "Datasets": { "description": "The list of 'Datasets' or data containers holding the actual data. The record ids are usually internal surrogate keys enabling the association of dataset records with work-product-component records, namely via 'DatasetIDs' and 'Artefacts.ResourceID' (both referring to 'dataset' group-type entity types).", "type": "array", "items": { "$ref": "GenericDataset.1.0.0.json" } } } } }, "x-osdu-inheriting-from-kind": [] } + validator._clear_data_fields(schema) + validator._validate_against_schema(schema, manifest) diff --git a/tests/plugin-unit-tests/test_search_record_id.py b/tests/plugin-unit-tests/test_search_record_id.py index b8003982447da9fe96ba1a7797b94d403f0e704c..d13938e9f5314fdf67b65d440dccfa5f08a43681 100644 --- a/tests/plugin-unit-tests/test_search_record_id.py +++ b/tests/plugin-unit-tests/test_search_record_id.py @@ -12,8 +12,7 @@ # 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 import http @@ -26,13 +25,13 @@ sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") from mock_providers import get_test_credentials from file_paths import ( SEARCH_INVALID_RESPONSE_PATH, - SEARCH_VALID_RESPONSE_PATH -) + SEARCH_VALID_RESPONSE_PATH, + SEARCH_EXTRACTED_IDS_PATH) from libs.exceptions import RecordsNotSearchableError from libs.context import Context from libs.refresh_token import AirflowTokenRefresher from tenacity import stop_after_attempt -from libs.search_record_ids import SearchId +from libs.search_record_ids import SearchId, ExtendedSearchId from mock_responses import MockSearchResponse @@ -133,3 +132,27 @@ class TestManifestProcessor: with pytest.raises(ValueError): SearchId("http://test", record_ids, AirflowTokenRefresher(get_test_credentials()), Context(app_key="", data_partition_id="")) + + @pytest.mark.parametrize( + "record_ids,search_response_path,extracted_ids_path", + [ + pytest.param( + ["test"], + SEARCH_VALID_RESPONSE_PATH, + SEARCH_EXTRACTED_IDS_PATH + ) + ] + ) + def test_search_found_all_records(self, monkeypatch, record_ids: list, + search_response_path: str, + extracted_ids_path: str): + self.mock_storage_response(monkeypatch, search_response_path, total_count=len(record_ids)) + with open(search_response_path) as f: + response = json.load(f) + id_searcher = ExtendedSearchId("http://test", record_ids, + AirflowTokenRefresher(get_test_credentials()), + Context(app_key="", data_partition_id="")) + record_ids = id_searcher._extract_id_from_response(response) + with open(extracted_ids_path) as f: + extracted_ids = json.load(f) + assert set(record_ids) == set(extracted_ids)