diff --git a/src/dags/default-ingest.py b/src/dags/default-ingest.py index fe4f30779af3a5b245f34120dfd9ffa4d5da25fc..7e0bfe21252245972d3993c1321ac4dedcb8ee31 100644 --- a/src/dags/default-ingest.py +++ b/src/dags/default-ingest.py @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -"""DAG for opaque ingestion""" +"""DAG for opaque ingestion.""" from datetime import timedelta @@ -25,10 +25,6 @@ from operators.update_status import UpdateStatusOperator from libs.create_records import create_records # isort:skip -""" -A workflow creating a record -""" - default_args = { "owner": "Airflow", "depends_on_past": False, @@ -39,10 +35,6 @@ default_args = { "retries": 0, "retry_delay": timedelta(minutes=5), "trigger_rule": "none_failed", - # 'queue': 'bash_queue', - # 'pool': 'backfill', - # 'priority_weight': 10, - # 'end_date': datetime(2016, 1, 1), } workflow_name = "Default_ingest" diff --git a/src/dags/libs/constants.py b/src/dags/libs/constants.py new file mode 100644 index 0000000000000000000000000000000000000000..e385809f4821e72fc3fedb78a58bdbf0106584a4 --- /dev/null +++ b/src/dags/libs/constants.py @@ -0,0 +1,20 @@ +# 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. + +"""Constants module.""" + +RETRIES = 3 +TIMEOUT = 1 +WAIT = 10 diff --git a/src/dags/libs/context.py b/src/dags/libs/context.py index c22d2d45b3360b8f80fe400b4a7ce6e9654b4b97..146edc356287f51a7056fef853420235341cfd89 100644 --- a/src/dags/libs/context.py +++ b/src/dags/libs/context.py @@ -13,23 +13,27 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Context module.""" import dataclasses @dataclasses.dataclass -class Context(object): - """ - Store data-partition-id and AppKey passed via Payload field of dagrun.conf. - """ +class Context: + """Class to store data-partition-id and AppKey.""" + data_partition_id: str app_key: str @classmethod def populate(cls, ctx: dict) -> 'Context': + """ + Populates Context dataclass from dagrun.conf dict. + + :return: populated Context + :rtype: Context + """ ctx_payload = ctx.pop('Payload') - ctx_obj = cls( - app_key=ctx_payload['AppKey'], - data_partition_id=ctx_payload['data-partition-id'] - ) + ctx_obj = cls(app_key=ctx_payload['AppKey'], + data_partition_id=ctx_payload['data-partition-id']) return ctx_obj diff --git a/src/dags/libs/create_records.py b/src/dags/libs/create_records.py index 76ae3c8f64fe4feafc6062dd9e904f697af8dbd1..d219677b34f9038fd82cf677ea58af590bab7cda 100644 --- a/src/dags/libs/create_records.py +++ b/src/dags/libs/create_records.py @@ -13,6 +13,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Provide R2 record processor.""" + import configparser import logging @@ -40,13 +42,24 @@ DEFAULT_VERSION = config.get("DEFAULTS", "kind_version") @refresh_token(AirflowTokenRefresher()) def create_update_record_request(headers, record_client, record): + """Send request to create or update records via RecordClient. + + :param headers: The request headers + :param record_client: RecordClient + :param record: The record to create or update + :return: Storage service response + """ resp = record_client.create_update_records([record], headers.items()) return resp def create_records(**kwargs): - # the only way to pass in values through the experimental api is through - # the conf parameter + """Create records. + The only way to pass in values through the experimental api is through + the conf parameter + + :return: response status + """ data_conf = kwargs["dag_run"].conf logger.debug(kwargs) diff --git a/src/dags/libs/exceptions.py b/src/dags/libs/exceptions.py index 839562f97723e06a29cd1ac67a465c1233f90f6d..78ef8d111490ca5c6407368a2542397c2525c021 100644 --- a/src/dags/libs/exceptions.py +++ b/src/dags/libs/exceptions.py @@ -13,71 +13,74 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Exceptions module.""" + class RecordsNotSearchableError(Exception): - """ - Raise when expected totalCount of records differs from actual one. - """ + """Raise when expected totalCount of records differs from actual one.""" pass class RefreshSATokenError(Exception): - """ - Raise when token is empty after attempt to get credentials from service account file. + """Raise when token is empty after attempt to get credentials from + service account file. """ pass class PipelineFailedError(Exception): - """ - Raise when pipeline failed. - """ + """Raise when pipeline failed.""" pass class EmptyManifestError(Exception): - """ - Raise when manifest field is empty. - """ + """Raise when manifest field is empty.""" pass + class GetSchemaError(Exception): - """ - Raise when can't find schema. - """ + """Raise when can't find schema.""" pass -class NotOSDUShemaFormatError(Exception): - """ - Raise when schema doesn't correspond OSDU format - """ + +class SRNNotFound(Exception): + """Raise when can't find SRN.""" pass + +class NotOSDUSchemaFormatError(Exception): + """Raise when schema doesn't correspond OSDU format.""" + pass + + class SAFilePathError(Exception): - """ - Raise when sa_file path is not specified in Env Variables. - """ + """Raise when sa_file path is not specified in Env Variables.""" pass + class FileSourceError(Exception): - """ - Raise when file doesn't exist under given path. - """ + """Raise when file doesn't exist under given path.""" pass + class GCSObjectURIError(Exception): - """ - Raise when wrong Google Storage Object was given. - """ + """Raise when wrong Google Storage Object was given.""" pass + class UploadFileError(Exception): - """ - Raise when there is an error while uploading a file into OSDU. - """ + """Raise when there is an error while uploading a file into OSDU.""" + class TokenRefresherNotPresentError(Exception): - """ - Raise when token refresher is not present in "refresh_token' decorator. - """ + """Raise when token refresher is not present in "refresh_token' decorator.""" pass + + +class NoParentEntitySystemSRNError(Exception): + """Raise when parent entity doesn't have system-generated SRN.""" + pass + + +class InvalidFileRecordData(Exception): + """Raise when file data does not contain mandatory fields.""" diff --git a/src/dags/libs/handle_file.py b/src/dags/libs/handle_file.py new file mode 100644 index 0000000000000000000000000000000000000000..c03ac40ece3d336d082190f326d7af8e1ad5e207 --- /dev/null +++ b/src/dags/libs/handle_file.py @@ -0,0 +1,363 @@ +# 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. + +"""This module provides cloud specific File Handler implementations.""" + +import dataclasses +import io +import json +import logging +import uuid +from abc import ABC, abstractmethod +from typing import List, Tuple, TypeVar +from urllib.parse import urlparse + +import requests +import tenacity +from google.cloud import storage +from libs.context import Context +from libs.constants import RETRIES, WAIT +from libs.exceptions import FileSourceError, GCSObjectURIError, InvalidFileRecordData +from libs.mixins import HeadersMixin +from libs.refresh_token import TokenRefresher, refresh_token + +logger = logging.getLogger() + +FileLikeObject = TypeVar("FileLikeObject", io.IOBase, io.RawIOBase, io.BytesIO) + +RETRY_SETTINGS = { + "stop": tenacity.stop_after_attempt(RETRIES), + "wait": tenacity.wait_fixed(WAIT), +} + + +@dataclasses.dataclass +class FileUploadUrlResponse: + """Simple class to store File service uploadURL response values.""" + file_id: str + signed_url: str + file_source: str + + +@dataclasses.dataclass +class FileDownloadUrlResponse: + """Simple class to store File service downloadURL response values.""" + signed_url: str + unsigned_url: str + kind: str + + +class FileHandler(HeadersMixin, ABC): + """Class to perform operations using OSDU File Service.""" + + def __init__(self, file_service_url: str, token_refresher: TokenRefresher, context: Context): + """File handler. + + :param file_service_url: Base OSDU File service url + :type file_service_url: str + :param token_refresher: Object to refresh tokens + :type token_refresher: TokenRefresher + :param context: The tenant context data + :type context: Context + """ + super().__init__(context) + self._file_service_url = file_service_url + self.token_refresher = token_refresher + + @abstractmethod + def get_file_from_preload_path(self, preload_path: str) -> Tuple[FileLikeObject, str]: + """Get a file from a given preload_path. + + :param preload_path: Full URI of the file to obtain + :type preload_path: str + :return: Raw file data and content-type + :rtype: Tuple[FileLikeObject, str] + """ + raise NotImplementedError("Provide an implementation according to cloud env.") + + @staticmethod + def _verify_file_record_data(file_record_data: dict): + """Perform simple verification of mandatory fields according to OSDU + File Service. + + :param file_record_data: Data field of file_record + :type file_record_data: dict + :raises InvalidFileRecordData: When some of the mandatory fields is + missing or empty + """ + endian = file_record_data.get("Endian") + file_source = file_record_data["DatasetProperties"]["FileSourceInfo"].get("FileSource") + if not (endian and file_source): + raise InvalidFileRecordData(f"Mandatory fields: Endian-{endian}" + f"FileSource-{file_source}") + + @tenacity.retry(**RETRY_SETTINGS) + @refresh_token() + def _send_post_request(self, headers: dict, url: str, request_body: str) -> requests.Response: + logger.debug(f"{request_body}") + response = requests.post(url, request_body, headers=headers) + logger.debug(response.content) + return response + + @tenacity.retry(**RETRY_SETTINGS) + @refresh_token() + def _send_get_request(self, headers: dict, url: str) -> requests.Response: + response = requests.get(url, headers=headers) + logger.debug(response) + return response + + def _get_upload_signed_url(self, headers: dict) -> FileUploadUrlResponse: + """Get FileID, SignedURL and FileSource using File Service uploadURL + endpoint. + + :param headers: Request headers to pass to the final request issuer + :type headers: dict + :return: FileUploadUrlResponse with data from service + :rtype: FileUploadUrlResponse + """ + logger.debug("Getting upload signed url.") + endpoint = f"{self._file_service_url}/v1/files/uploadURL" + response = self._send_get_request(headers, endpoint).json() + logger.debug("Signed url got.") + upload_url_response = FileUploadUrlResponse(file_id=response["FileID"], + signed_url=response["Location"]["SignedURL"], + file_source=response["Location"]["FileSource"]) + return upload_url_response + + def _get_download_signed_url(self, headers: dict, record_id: str) -> FileDownloadUrlResponse: + """Get signedURL, unsignedURL and kind using File Service downloadURL + endpoint. + + :param headers: Request headers to pass to the final request issuer + :type headers: dict + :param record_id: Unique id of the file record saved in the osdu system + :type record_id: str + :return: FileDownloadUrlResponse with signed and unsigned urls + :rtype: FileDownloadUrlResponse + """ + logger.debug("Getting download signed url.") + endpoint = f"{self._file_service_url}/v1/files/{record_id}/downloadURL" + response = self._send_get_request(headers, endpoint).json() + logger.debug("Signed url got.") + download_url_response = FileDownloadUrlResponse(signed_url=response["signedUrl"], + unsigned_url=response["unsignedUrl"], + kind=response["kind"]) + return download_url_response + + @tenacity.retry(**RETRY_SETTINGS) + def _upload_file_request(self, headers: dict, signed_url: str, buffer: FileLikeObject): + """Upload file via File service using signed_url. + + :param headers: Request headers to pass to the final request issuer + :type headers: dict + :param signed_url: SignedURL to authenticate request + :type signed_url: str + :param buffer: Raw file data + :type buffer: FileLikeObject + """ + logger.debug("Uploading file.") + buffer.seek(0) + requests.put(signed_url, buffer.read(), headers=headers) + logger.debug("File uploaded.") + + def _get_file_location_request(self, headers: dict, file_id: str) -> str: + """Get file location using File Service. + + :param headers: Request headers to pass to the final request issuer + :type headers: dict + :param file_id: String identifier of the file + :type file_id: str + :return: Full URI of the located file + :rtype: str + """ + logger.debug("Getting file location.") + request_body = json.dumps({"FileID": file_id}) + endpoint = f"{self._file_service_url}/getFileLocation" + response = self._send_post_request(headers, endpoint, request_body) + logger.debug("File location got.") + return response.json()["Location"] + + @tenacity.retry(**RETRY_SETTINGS) + def upload_file(self, preload_file_path: str) -> str: + """Upload file from preload_file_path to Landing Zone (staging area) + onto OSDU platform using File service. + + :param preload_file_path: Full URI of the file to obtain + :type preload_file_path: str + :return: The FileSource (relative URI) of the uploaded file + :rtype: str + """ + buffer = self.get_file_from_preload_path(preload_file_path) + upload_url_response = self._get_upload_signed_url(self.request_headers) + self._upload_file_request(self.request_headers, upload_url_response.signed_url, buffer) + + return upload_url_response.file_source + + def get_file_staging_location(self, file_source: str) -> str: + """Retrieve location (full URI) of the file in staging area. + + :param file_source: The FileSource (relative URI) of the file of the form + /{folder}/{file_id} + :type file_source: str + :return: Full URI of the location of the file in staging area + :rtype: str + """ + file_id = file_source.split("/")[-1] + file_staging_location = self._get_file_location_request(self.request_headers, file_id) + return file_staging_location + + def get_file_permanent_location(self, file_record_id: str) -> str: + """Retrieve location (full URI) of the file in permanent area. + + :param file_record_id: The unique id of the file record (aka metadata + :type file_record_id: str + :return: Full URI of the location of the file in permanent area + :rtype: str + """ + download_url_response = self._get_download_signed_url(self.request_headers, file_record_id) + permanent_location = download_url_response.unsigned_url + return permanent_location + + def save_file_record(self, file_record: dict) -> str: + """Send request to store record via file service API. + + :param file_record: The file record to save + :type file_record: dict + :return: OSDU system generated id of the saved record + :rtype: str + """ + self._verify_file_record_data(file_record["data"]) + # TODO fix 'name' field processing + # Generate file entity name as workaround because file API required this field. + if not file_record["data"].get("Name"): + file_record["data"]["Name"] = \ + f"surrogate_name_{file_record['data']['DatasetProperties']['FileSourceInfo']['PreloadFilePath'].split('/')[-1]}" + logger.info(f"Generated name: {file_record['data']['Name']}") + logger.info("Sending file record metadata to File service") + endpoint = f"{self._file_service_url}/v1/files/metadata" + response = self._send_post_request(self.request_headers, endpoint, json.dumps(file_record)) + + return response.json()["id"] + + def batch_save_file_records(self, file_records: List[str]) -> List[str]: + """Perform concurrent save file record requests. + + :param file_records: List of file records to save + :type file_records: List[str] + :return: List of OSDU system generated ids of the saved records + :rtype: List[str] + """ + raise NotImplementedError("TODO(python-team) implementation.") + + +class GCSFileHandler(FileHandler): + """GCP specific implementation of FileHandler.""" + + def __init__( + self, + file_service_url: str, + token_refresher: TokenRefresher, + context: Context, + ): + """GCS specific FileHandler. + + :param file_service_url: Base osdu file service url + :type file_service_url: str + :param token_refresher: Object to refresh tokens + :type token_refresher: TokenRefresher + :param context: The tenant context data + :type context: Context + """ + super().__init__(file_service_url, token_refresher, context) + + @staticmethod + def _parse_object_uri(file_path: str) -> Tuple[str, str]: + """Parse a URI to obtain the bucket name and the blob name. + + :param file_path: URI to parse + :type file_path: str + :raises GCSObjectURIError: When URI is not compliant with GCS + :return: The bucket and blob names + :rtype: Tuple[str, str] + """ + parsed_path = urlparse(file_path) + if parsed_path.scheme == "gs": + bucket_name = parsed_path.netloc + source_blob_name = parsed_path.path[1:] # delete the first slash + + if bucket_name and source_blob_name: + return bucket_name, source_blob_name + + raise GCSObjectURIError + + @tenacity.retry(**RETRY_SETTINGS) + def get_file_from_bucket(self, + bucket_name: str, + source_blob_name: str) -> Tuple[io.BytesIO, str]: + """Get file and content type using GCS client. + + :param bucket_name: The name of the bucket + :type bucket_name: str + :param source_blob_name: The name of the file + :type source_blob_name: str + :raises FileSourceError: When validation of file fails + :return: The raw data of the file and the content-type + :rtype: Tuple[io.BytesIO, str] + """ + storage_client = storage.Client() + bucket = storage_client.bucket(bucket_name) + blob = bucket.get_blob(source_blob_name) + + does_exist = blob.exists() + if not does_exist: + raise FileSourceError("File doesn't exist in preloadPath " + f"'gs://{bucket_name}/{source_blob_name}'") + + file = io.BytesIO() + blob.download_to_file(file) + logger.debug("File got from preload_path zone") + return file, blob.content_type + + def get_file_from_preload_path(self, preload_file_path: str) -> Tuple[io.BytesIO, str]: + """Get file from given preload file path. + + :param preload_file_path: The full URI of the preloaded file + :type preload_file_path: str + :return: The raw data of the file and the content-type + :rtype: Tuple[io.BytesIO, str] + """ + bucket_name, blob_name = self._parse_object_uri(preload_file_path) + buffer, content_type = self.get_file_from_bucket(bucket_name, blob_name) + return buffer, content_type + + def upload_file(self, preload_file_path: str) -> str: + """Copy file from preload_file_path location to Landing Zone in OSDU + platform using File service. Get Content-Type of this file, refresh + Content-Type with this value in headers while this file is being + uploaded onto OSDU platform. + + :param preload_file_path: The URI of the preloaded file + :type preload_file_path: str + :return: FileSource obtained via File service + :rtype: str + """ + buffer, content_type = self.get_file_from_preload_path(preload_file_path) + upload_url_response = self._get_upload_signed_url(self.request_headers) + + headers = self.request_headers + headers["Content-Type"] = content_type + self._upload_file_request(headers, upload_url_response.signed_url, buffer) + + return upload_url_response.file_source diff --git a/src/dags/libs/mixins.py b/src/dags/libs/mixins.py index 7289a59981f5030867e5bfc89705fe06b96d249a..a0cf661b07993b9b3615d4e3ab51aa5d406ecc7b 100644 --- a/src/dags/libs/mixins.py +++ b/src/dags/libs/mixins.py @@ -13,20 +13,29 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Mixins.""" from libs.context import Context class HeadersMixin(object): - """ - Mixin for creating request headers to OSDU services using context. - """ + """Mixin for creating request headers to OSDU services using context.""" def __init__(self, context: Context): + """Headers Mixin. + + :param context: Context dataclass that contains tenant information + :type context: Context + """ self.context = context @property def request_headers(self) -> dict: + """Default request headers. + + :return: headers dict populated from Context + :rtype: dict + """ headers = { 'Content-type': 'application/json', 'data-partition-id': self.context.data_partition_id, diff --git a/src/dags/libs/process_manifest_r3.py b/src/dags/libs/process_manifest_r3.py index 3e0cf5157777e199282191f264b3a54d7632646d..6b509aba8d0a8df1d6fa704bc88be141852eec77 100644 --- a/src/dags/libs/process_manifest_r3.py +++ b/src/dags/libs/process_manifest_r3.py @@ -13,21 +13,29 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Provides Manifest Processor.""" + import copy import json import logging import uuid -from typing import List +from typing import List, Tuple import requests import tenacity from libs.context import Context -from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError -from libs.source_file_check import SourceFileChecker -from libs.upload_file import FileUploader +from libs.constants import RETRIES, WAIT +from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError from libs.mixins import HeadersMixin -from libs.refresh_token import AirflowTokenRefresher, refresh_token, TokenRefresher +from libs.refresh_token import TokenRefresher, refresh_token +from libs.source_file_check import SourceFileChecker +from libs.handle_file import FileHandler +RETRY_SETTINGS = { + "stop": tenacity.stop_after_attempt(RETRIES), + "wait": tenacity.wait_fixed(WAIT), + "reraise": True +} logger = logging.getLogger() RETRIES = 3 @@ -35,49 +43,77 @@ TIMEOUT = 1 class ManifestProcessor(HeadersMixin): - """Class to process WP, Master and Reference data""" + """Class to process WP, Master and Reference data.""" + RECORD_TEMPLATE = { "legal": {}, "acl": {}, "kind": "", "id": "", - "data": { - } + "data": {} } def __init__( self, storage_url: str, - dagrun_conf: dict, - file_uploader: FileUploader, + manifest_records: List[dict], + file_handler: FileHandler, source_file_checker: SourceFileChecker, token_refresher: TokenRefresher, context: Context ): + """Manifest processor. + + :param storage_url: The OSDU Storage base url + :type storage_url: str + :param dagrun_conf: The conf obtained from dagrun + :type dagrun_conf: dict + :param file_handler: An instance of a file handler + :type file_handler: FileHandler + :param source_file_checker: An instance of file checker + :type source_file_checker: SourceFileChecker + :param token_refresher: An instance of token refresher + :type token_refresher: TokenRefresher + :param context: The tenant context + :type context: Context + """ super().__init__(context) - self.file_uploader = file_uploader + self.file_handler = file_handler self.source_file_checker = source_file_checker self.storage_url = storage_url - self.data_object = copy.deepcopy(dagrun_conf) + self.manifest_records = manifest_records self.context = context self.token_refresher = token_refresher @staticmethod def _get_kind_name(kind: str) -> str: - """ - osdu:osdu:Well:1.0.0 -> Well - """ + """Get and return kind name. Ex., osdu:osdu:Well:1.0.0 -> Well.""" kind_name = kind.split(":")[2] return kind_name def upload_source_file(self, file_record: dict) -> dict: - file_path = file_record["data"]["PreLoadFilePath"] - file_record["data"]["FileSource"] = self.file_uploader.upload_file(file_path) + """Upload files from preloadfile path. + + :param file_record: The file record + :type file_record: dict + :return: file record updated if file was properly uploaded + :rtype: dict + """ + file_path = file_record["data"]["DatasetProperties"]["FileSourceInfo"]["PreloadFilePath"] + try: + file_source = self.file_handler.upload_file(file_path) + file_record["data"]["DatasetProperties"]["FileSourceInfo"]["FileSource"] = file_source + except Exception as e: + logger.error(f"Unhandled exception while uploading {file_path}: {e}") return file_record def generate_id(self, manifest_fragment: dict) -> str: - """ - Generate id to use it in Storage. + """Generate id to use it in Storage. + + :param manifest_fragment: The manifest to update + :type manifest_fragment: dict + :return: The generated id + :rtype: str """ group_type = manifest_fragment.get("groupType", "doc") kind = manifest_fragment.get("kind") @@ -86,38 +122,51 @@ class ManifestProcessor(HeadersMixin): return _id def populate_manifest_storage_record(self, manifest: dict) -> dict: - """ - Create a record from Master-manifest to store it in Storage service + """Create a record from manifest to store it in Storage service. + + :param manifest: The manifest to populate from + :type manifest: dict + :return: The populated record + :rtype: dict """ record = copy.deepcopy(self.RECORD_TEMPLATE) record["id"] = manifest["id"] if manifest.get("id") else self.generate_id(manifest) record["kind"] = manifest.pop("kind") record["legal"] = manifest.pop("legal") record["acl"] = manifest.pop("acl") - record["data"] = manifest + record["data"] = manifest.pop("data") return record - @staticmethod - def _validate_file_source_checker_type(file_source_checker: SourceFileChecker): - if not isinstance(file_source_checker, SourceFileChecker): - raise TypeError(f"File checker must be instance of '{SourceFileChecker}' class.\n" - f"Got got instance of '{file_source_checker}'") + def _populate_file_storage_record(self, manifest: dict) -> dict: + """Create a record from file manifest to store it via File service. + + :param manifest: file manifest + :type manifest: dict + :return: file record + :rtype: dict + """ + record = copy.deepcopy(self.RECORD_TEMPLATE) + record["id"] = manifest["id"] if manifest.get("id") else self.generate_id(manifest) + record["kind"] = manifest.pop("kind") + record["legal"] = manifest.pop("legal") + record["acl"] = manifest.pop("acl") + record["data"] = manifest.pop("data") + # TODO(python-team) verify that no data goes under data. + # then update just -> record["data"].update(manifest) + return record @staticmethod def _validate_storage_response(response_dict: dict): + """Validate Storage service response.""" if not ( isinstance(response_dict, dict) and isinstance(response_dict.get("recordIds"), list) ): raise ValueError(f"Invalid answer from Storage server: {response_dict}") - @tenacity.retry( - wait=tenacity.wait_fixed(TIMEOUT), - stop=tenacity.stop_after_attempt(RETRIES), - reraise=True - ) - @refresh_token(AirflowTokenRefresher()) - def save_record(self, headers: dict, request_data: List[dict]) -> requests.Response: + @tenacity.retry(**RETRY_SETTINGS) + @refresh_token() + def save_record_to_storage(self, headers: dict, request_data: List[dict]) -> requests.Response: """ Send request to record storage API. """ @@ -138,71 +187,53 @@ class ManifestProcessor(HeadersMixin): f"Response content: {reason}.") return response - def process_work_product(self, manifest: dict) -> List[dict]: + def save_record_to_file_service(self, file_records: List[dict]) -> List[str]: """ - Process WP. + Send request to file service API """ - wp = manifest["WorkProduct"] - records = [self.populate_manifest_storage_record(wp)] - return records + file_record_ids = [] + for file_record in file_records: + # TODO(python-team) implement concurrent request in File Handler service. + record_id = self.file_handler.save_file_record(file_record) + file_location = self.file_handler.get_file_permanent_location(record_id) + # TODO(python-team) implement rollback strategy in case file validation fails. + self.source_file_checker.does_file_exist(file_location) + file_record_ids.append(record_id) + return file_record_ids - def process_work_product_components(self, manifest: dict) -> List[dict]: + def process_work_product_files(self, file_records: List[dict]) -> List[dict]: """ - Process list of WPS. + Process list of file records. """ records = [] - for wpc in manifest["WorkProductComponents"]: - record = self.populate_manifest_storage_record(wpc) - records.append(record) - return records - - def process_work_product_files(self, manifest: dict) -> List[dict]: - """ - Process list of files. - """ - records = [] - for file in manifest["Files"]: - if not file["data"]["FileSource"]: - file = self.upload_source_file(file) + for file_record in file_records: + if not file_record["data"]["DatasetProperties"]["FileSourceInfo"]["FileSource"]: + file_record = self.upload_source_file(file_record) else: - self.source_file_checker.does_file_exist(file["data"]["FileSource"]) + file_source = file_record["data"]["DatasetProperties"]["FileSourceInfo"]["FileSource"] + file_location = self.file_handler.get_file_staging_location(file_source) + self.source_file_checker.does_file_exist(file_location) - record = self.populate_manifest_storage_record(file) + record = self._populate_file_storage_record(file_record) records.append(record) return records - def process_work_product_manifest(self, manifest: dict) -> List[dict]: - file_records = self.process_work_product_files(manifest) - wpc_records = self.process_work_product_components(manifest) - wp_records = self.process_work_product(manifest) - records = file_records + wpc_records + wp_records - return records - - def create_manifest_records(self) -> List[dict]: - """ - Process every record in manifest field - Return list of records ready to be saved into Storage service. - """ - manifest_records = [] - manifests = self.data_object["manifest"] - for manifest in manifests: - if "WorkProduct" in manifest: - wp_records = self.process_work_product_manifest(manifest) - manifest_records.extend(wp_records) - else: - record = self.populate_manifest_storage_record(manifest) - manifest_records.append(record) - return manifest_records - def process_manifest(self) -> List[str]: + """Process manifests and save them into Storage service. + + :raises EmptyManifestError: When manifest is empty + :return: List of ids of saved records + :rtype: List[str] """ - Process manifests and save them into Storage service. - Returns recordIds of saved records. - """ - if "manifest" in self.data_object: - manifest_records = self.create_manifest_records() - else: + record_ids = [] + populated_manifest_records = [] + if not self.manifest_records: raise EmptyManifestError - response = self.save_record(self.request_headers, request_data=manifest_records) - record_ids = response.json()["recordIds"] + for manifest_record in self.manifest_records: + populated_manifest_records.append(self.populate_manifest_storage_record(manifest_record.get("entity"))) + save_manifests_response = self.save_record_to_storage( + self.request_headers, request_data=populated_manifest_records) + record_ids.extend(save_manifests_response.json()["recordIds"]) + + logger.warning(record_ids) return record_ids diff --git a/src/dags/libs/refresh_token.py b/src/dags/libs/refresh_token.py index dc0a2eba93e7730f2490967c821f2edf9e13f7f8..cb3e5e91c6592d1ecdbc291b20b68709bcc0043c 100644 --- a/src/dags/libs/refresh_token.py +++ b/src/dags/libs/refresh_token.py @@ -13,6 +13,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Auth and refresh token utility functions.""" + import json import logging import os @@ -35,25 +37,36 @@ RETRIES = 3 class TokenRefresher(ABC): + """Abstract base class for token refreshers.""" @abstractmethod def refresh_token(self) -> str: - """ - Implement logics of refreshing token here. + """Refresh auth token. + + :return: refreshed token + :rtype: str """ pass @property @abstractmethod def access_token(self) -> str: + """Auth access token. + + :return: token string + :rtype: str + """ pass @property @abstractmethod def authorization_header(self) -> dict: - """ - Must return authorization header for updating headers dict. + """Authorization header. Must return authorization header for + updating headers dict. E.g. return {"Authorization": "Bearer <access_token>"} + + :return: A dictionary with authorization header updated + :rtype: dict """ pass @@ -62,6 +75,12 @@ class AirflowTokenRefresher(TokenRefresher): DEFAULT_ACCESS_SCOPES = ['openid', 'email', 'profile'] def __init__(self, access_scopes: list = None): + """Init GCP Token Refresher. + + :param access_scopes: the list of scopes for authorization, defaults + to None + :type access_scopes: list, optional + """ from airflow.models import Variable self.airflow_variables = Variable self._access_token = None @@ -69,9 +88,11 @@ class AirflowTokenRefresher(TokenRefresher): @property def access_scopes(self) -> list: - """ - Return access scopes. - Use DEFAULT_ACCESS_SCOPES if user-defined ones weren't provided. + """Return access scopes. Use DEFAULT_ACCESS_SCOPES if user-defined + ones weren't provided. + + :return: A list of scopes to authorize + :rtype: list """ if not self._access_scopes: self._access_scopes = self.DEFAULT_ACCESS_SCOPES @@ -80,8 +101,14 @@ class AirflowTokenRefresher(TokenRefresher): @staticmethod @retry(stop=stop_after_attempt(RETRIES)) def get_sa_info_from_google_storage(bucket_name: str, source_blob_name: str) -> dict: - """ - Get sa_file content from Google Storage. + """Get service account file content from Google Storage. + + :param bucket_name: bucket where file is located + :type bucket_name: str + :param source_blob_name: name of the service account key file + :type source_blob_name: str + :return: A dictionary with the information loaded from file + :rtype: dict """ storage_client = storage.Client() bucket = storage_client.bucket(bucket_name) @@ -92,14 +119,24 @@ class AirflowTokenRefresher(TokenRefresher): @staticmethod def get_sa_info_from_file(path: str) -> dict: + """Load service account information from file. + + :param path: file path + :type path: str + :return: A dictionary with the information loaded from file + :rtype: dict + """ with open(path) as f: return json.load(f) def get_sa_info(self) -> dict: - """ - Get file path from SA_FILE_PATH environmental variable. - This path can be GCS object URI or local file path. - Return content of sa path as dict. + """Get file path from SA_FILE_PATH environment variable. This path + can be GCS object URI or local file path. Return content of sa path + as dict. + + :raises SAFilePathError: When a invalid path is retrieved from env + :return: A dictionary with the information loaded from file + :rtype: dict """ sa_file_path = os.environ.get("SA_FILE_PATH", None) parsed_path = urlparse(sa_file_path) @@ -116,6 +153,14 @@ class AirflowTokenRefresher(TokenRefresher): @retry(stop=stop_after_attempt(RETRIES)) def _get_credentials_from_sa_info(self, sa_info: dict) -> service_account.Credentials: + """Get credentials from service account info. + + :param sa_info: The service account info as dict + :type sa_info: dict + :raises e: Generic exception + :return: Google service account credentials + :rtype: service_account.Credentials + """ try: credentials = service_account.Credentials.from_service_account_info( sa_info, scopes=self.access_scopes) @@ -125,8 +170,11 @@ class AirflowTokenRefresher(TokenRefresher): return credentials def get_access_token_using_sa_file(self) -> str: - """ - Get new access token using SA info. + """Get new access token using SA info. + + :raises RefreshSATokenError: When token can't be obtained after refresh + :return: The refreshed access token + :rtype: str """ sa_info = self.get_sa_info() credentials = self._get_credentials_from_sa_info(sa_info) @@ -144,8 +192,10 @@ class AirflowTokenRefresher(TokenRefresher): @retry(stop=stop_after_attempt(RETRIES)) def refresh_token(self) -> str: - """ - Refresh token. Store its value in Airflow Variable. + """Refresh token. Also, it stores its value in Airflow Variable. + + :return: The refreshed access token + :rtype: str """ token = self.get_access_token_using_sa_file() self.airflow_variables.set("access_token", token) @@ -154,6 +204,11 @@ class AirflowTokenRefresher(TokenRefresher): @property def access_token(self) -> str: + """Access token. + + :return: The access token + :rtype: str + """ if not self._access_token: try: self._access_token = self.airflow_variables.get("access_token") @@ -163,13 +218,26 @@ class AirflowTokenRefresher(TokenRefresher): @property def authorization_header(self) -> dict: + """Authorizatioh header. + + :return: A dict with the authorization header updated + :rtype: dict + """ return {"Authorization": f"Bearer {self.access_token}"} def make_callable_request(obj: Union[object, None], request_function: Callable, headers: dict, *args, **kwargs) -> Callable: - """ - Create send_request_with_auth function. + """Create send_request_with_auth function. + + :param obj: If wrapping a method the obj passed as first argument (self) + :type obj: Union[object, None] + :param request_function: The function to be build + :type request_function: Callable + :param headers: The request headers + :type headers: dict + :return: A partial callable + :rtype: Callable """ if obj: # if wrapped function is an object's method callable_request = partial(request_function, obj, headers, *args, **kwargs) @@ -203,9 +271,16 @@ def _get_object_token_refresher( token_refresher: TokenRefresher, obj: object = None ) -> TokenRefresher: - """ - Check if token refresher passed into decorator or specified in object's as 'token_refresher' - property. + """Get token refresher passed into decorator or specified in object's as + 'token_refresher' property. + + :param token_refresher: A token refresher instance + :type token_refresher: TokenRefresher + :param obj: The holder object of the decorated method, defaults to None + :type obj: object, optional + :raises TokenRefresherNotPresentError: When not found + :return: The token refresher + :rtype: TokenRefresher """ if token_refresher is None and obj: try: @@ -220,9 +295,15 @@ def _get_object_token_refresher( def send_request_with_auth_header(token_refresher: TokenRefresher, *args, **kwargs) -> requests.Response: - """ - Send request with authorization token. If response status is in HTTPStatus.UNAUTHORIZED or - HTTPStatus.FORBIDDEN, then refresh token and send request once again. + """Send request with authorization token. If response status is in + HTTPStatus.UNAUTHORIZED or HTTPStatus.FORBIDDEN, then refreshes token + and sends request once again. + + :param token_refresher: The token refresher instance + :type token_refresher: TokenRefresher + :raises e: Re-raises any requests.HTTPError + :return: The server response + :rtype: requests.Response """ obj = kwargs.pop("obj", None) request_function = kwargs.pop("request_function") @@ -252,14 +333,19 @@ def send_request_with_auth_header(token_refresher: TokenRefresher, *args, def refresh_token(token_refresher: TokenRefresher = None) -> Callable: - """ - Wrap a request function and check response. If response's error status code - is about Authorization, refresh token and invoke this function once again. - Expects function: + """Wrap a request function and check response. If response's error status + code is about Authorization, refresh token and invoke this function once + again. + Expects function. If response is not ok and not about Authorization, then raises HTTPError request_func(header: dict, *args, **kwargs) -> requests.Response Or method: request_method(self, header: dict, *args, **kwargs) -> requests.Response + + :param token_refresher: [description], defaults to None + :type token_refresher: TokenRefresher, optional + :return: [description] + :rtype: Callable """ def refresh_token_wrapper(request_function: Callable) -> Callable: diff --git a/src/dags/libs/search_record_ids.py b/src/dags/libs/search_record_ids.py index e33a5ca1629bd9477769f2f2794e47f1914606ac..e45a0287ef4d3f79ffc0d90d958b0a4872eb1705 100644 --- a/src/dags/libs/search_record_ids.py +++ b/src/dags/libs/search_record_ids.py @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Provides SearchId validator.""" import json import logging @@ -22,7 +23,7 @@ import tenacity from libs.context import Context from libs.exceptions import RecordsNotSearchableError from libs.mixins import HeadersMixin -from libs.refresh_token import AirflowTokenRefresher, refresh_token +from libs.refresh_token import TokenRefresher, refresh_token logger = logging.getLogger() @@ -32,8 +33,22 @@ TIMEOUT = 10 class SearchId(HeadersMixin): + """Class for stored records search validation.""" - def __init__(self, search_url: str, record_ids: list, token_refresher, context: Context): + def __init__(self, search_url: str, record_ids: list, token_refresher: TokenRefresher, + context: Context): + """Init search id validator. + + :param search_url: Base OSDU Search service url + :type search_url: str + :param record_ids: The list of records id to be searched + :type record_ids: list + :param token_refresher: An instance of token refresher + :type token_refresher: Token Refresher + :param context: The tenant context + :type context: Context + :raises ValueError: When a mismatch of record counts occur + """ super().__init__(context) if not record_ids: logger.error("There are no record ids") @@ -45,8 +60,11 @@ class SearchId(HeadersMixin): self._create_request_body() def _create_search_query(self) -> str: - """ - Create search query to send to Search service using recordIds need to be found. + """Create search query to send to Search service using recordIds need + to be found. + + :return: An elastic search valid query + :rtype: str """ record_ids = " OR ".join(f"\"{id_}\"" for id_ in self.record_ids) logger.debug(f"Search query {record_ids}") @@ -54,9 +72,7 @@ class SearchId(HeadersMixin): return query def _create_request_body(self): - """ - Create request body to send to Search service. - """ + """Create request body to send to Search service.""" query = self._create_search_query() request_body = { "kind": "*:*:*:*", @@ -65,8 +81,13 @@ class SearchId(HeadersMixin): self.request_body = json.dumps(request_body) def _is_record_searchable(self, response: requests.Response) -> bool: - """ - Check if search service returns expected totalCount of records. + """Check if search service returns expected totalCount of records. + + :param response: The response returned from the Search service + :type response: requests.Response + :raises ValueError: When a mismatch of record counts occur + :return: [description] + :rtype: bool """ logger.debug(response.text) data = response.json() @@ -84,8 +105,14 @@ class SearchId(HeadersMixin): ) @refresh_token() def search_files(self, headers: dict) -> requests.Response: - """ - Send request with recordIds to Search service. + """Send request with recordIds to Search service. + + :param headers: The request headers + :type headers: dict + :raises RecordsNotSearchableError: When any of the records is not + searchable + :return: The server response + :rtype: requests.Response """ if self.request_body: response = requests.post(self.search_url, self.request_body, headers=headers) @@ -100,8 +127,6 @@ class SearchId(HeadersMixin): return response def check_records_searchable(self): - """ - Check if every record in self.record_ids is searchable. - """ + """Check if every record in self.record_ids is searchable.""" headers = self.request_headers self.search_files(headers) diff --git a/src/dags/libs/source_file_check.py b/src/dags/libs/source_file_check.py index 6f5468fa42f9648968e14f5f7d9957cedff4a5a8..6f1a97595f9542ccb547a185aa99bd504bb6e8e5 100644 --- a/src/dags/libs/source_file_check.py +++ b/src/dags/libs/source_file_check.py @@ -13,11 +13,12 @@ # See the License for the specific language governing permissions and # limitations under the License. - +import logging from abc import ABC, abstractmethod from typing import Tuple from urllib.parse import urlparse +import google.auth import tenacity from google.cloud import storage from libs.exceptions import GCSObjectURIError, FileSourceError @@ -25,29 +26,35 @@ from libs.exceptions import GCSObjectURIError, FileSourceError RETRIES = 3 +logger = logging.getLogger() + class SourceFileChecker(ABC): + """Class to verify the existance of a file in cloud storage.""" @abstractmethod - def does_file_exist(self, file_path: str) -> bool: - """ - Validate if file exists under given file path. + def does_file_exist(self, file_path: str): + """Validate if file exists under given file path. Raises error if not. + + :param file_path: The full URI if the file to verify + :type file_path: str """ pass class GCSSourceFileChecker(SourceFileChecker): - def __init__(self): - pass - def __repr__(self): return "GCS file checker" def _parse_object_uri(self, file_path: str) -> Tuple[str, str]: - """ - Parse GCS Object uri. - Return bucket and blob names. + """Parse GCS Object uri. + + :param file_path: The URI to parse + :type file_path: str + :raises GCSObjectURIError: When Tthe format of the URI does not comply + :return: The bucket and the blob names + :rtype: Tuple[str, str] """ parsed_path = urlparse(file_path) if parsed_path.scheme == "gs": @@ -64,15 +71,34 @@ class GCSSourceFileChecker(SourceFileChecker): reraise=True ) def _does_file_exist_in_bucket(self, bucket_name: str, source_blob_name: str) -> bool: + """Do the actual verification of file via gcs client. + + :param bucket_name: The name of the bucket + :type bucket_name: str + :param source_blob_name: The name of the file + :type source_blob_name: str + :return: A boolean indicating if the file exists + :rtype: bool + """ storage_client = storage.Client() bucket = storage_client.bucket(bucket_name) blob = bucket.blob(source_blob_name) does_exist = blob.exists() return does_exist - def does_file_exist(self, file_path: str) -> bool: + def does_file_exist(self, file_path: str): + """Verify if file exists under given file path (URI). + + :param file_path: The full URI of the file to verify + :type file_path: str + :raises FileSourceError: When the file does not exists + """ bucket_name, source_blob_name = self._parse_object_uri(file_path) - does_exist = self._does_file_exist_in_bucket(bucket_name, source_blob_name) - if not does_exist: - raise FileSourceError(f"File doesn't exist in '{file_path}'") - return True + try: + does_exist = self._does_file_exist_in_bucket(bucket_name, source_blob_name) + except google.auth.exceptions.DefaultCredentialsError: + # TODO(python-team) Figure out a way to mock google endpoints in integration tests. + logger.error("No default credentials found in env, is this integration-env?") + else: + if not does_exist: + raise FileSourceError(f"File doesn't exist in {file_path}.") diff --git a/src/dags/libs/traverse_manifest.py b/src/dags/libs/traverse_manifest.py new file mode 100644 index 0000000000000000000000000000000000000000..510cb5f22b1812abfac77f96df149248dd3be96b --- /dev/null +++ b/src/dags/libs/traverse_manifest.py @@ -0,0 +1,90 @@ +# 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 logging +from typing import List + +from libs.exceptions import EmptyManifestError + + +logger = logging.getLogger() + + +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) + self.manifest_schema = manifest_schema + self.manifest_info = [] + + def _populate_manifest_entity(self, entity: dict, schema: str): + """ + Populate manifest entity for future processing + + :param entity: manifest entity instance (for future processing) + :param schema: corresponding generic schema (for future schema validation) + :return: + """ + return { + "schema": schema, + "entity": entity + } + + def _traverse_list(self, manifest_entities: List[dict], property_name: str, manifest_schema_part: dict): + """ + Traverse list of entities and returned populated list of entities + """ + entities = [] + for manifest_entity in manifest_entities: + entities.append( + self._populate_manifest_entity(manifest_entity, + manifest_schema_part[property_name]["items"]["$ref"])) + return entities + + def traverse_manifest(self) -> List[dict]: + """ + Traverse manifest structure and return the list of manifest records. + + :return: list of records + """ + if "manifest" not in self.data_object: + raise EmptyManifestError + + manifest_entities = [] + manifest_file = self.data_object["manifest"] + if manifest_file.get("ReferenceData"): + manifest_entities.extend(self._traverse_list(manifest_file["ReferenceData"], + "ReferenceData", + self.manifest_schema["properties"])) + if manifest_file.get("MasterData"): + manifest_entities.extend(self._traverse_list(manifest_file["MasterData"], + "MasterData", + self.manifest_schema["properties"])) + if manifest_file.get("Data"): + if manifest_file["Data"].get("WorkProduct"): + manifest_entities.append(self._populate_manifest_entity( + manifest_file["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"], + "WorkProductComponents", + self.manifest_schema["properties"]["Data"]["properties"])) + if manifest_file["Data"].get("Datasets"): + manifest_entities.extend(self._traverse_list(manifest_file["Data"]["Datasets"], + "Datasets", + self.manifest_schema["properties"]["Data"]["properties"])) + return manifest_entities diff --git a/src/dags/libs/update_status.py b/src/dags/libs/update_status.py index 5b756747437dc8219c29e6270aac8ba91e139fc4..3e44375e0f11ae2c60a004cf4b0188449ba1aaf2 100644 --- a/src/dags/libs/update_status.py +++ b/src/dags/libs/update_status.py @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Provides UpdateStatus processor.""" import json import logging @@ -26,6 +27,7 @@ logger = logging.getLogger() class UpdateStatus(HeadersMixin): + """Class to perform update status of the workflow.""" def __init__( self, @@ -35,6 +37,19 @@ class UpdateStatus(HeadersMixin): token_refresher: TokenRefresher, context: Context, ) -> None: + """Init the status update processor. + + :param workflow_id: The id of the workflow + :type workflow_id: str + :param workflow_url: The base url of the Workflow service + :type workflow_url: str + :param status: The status + :type status: str + :param token_refresher: An instance of token refresher + :type token_refresher: TokenRefresher + :param context: The tenant context + :type context: Context + """ super().__init__(context) self.workflow_url = workflow_url self.workflow_id = workflow_id @@ -44,6 +59,13 @@ class UpdateStatus(HeadersMixin): @refresh_token() def update_status_request(self, headers: dict) -> requests.Response: + """Send request to update status. + + :param headers: The request headers + :type headers: dict + :return: The Workflow server response + :rtype: requests.Response + """ request_body = { "WorkflowID": self.workflow_id, "Status": self.status @@ -54,8 +76,6 @@ class UpdateStatus(HeadersMixin): return response def update_workflow_status(self): - """ - Update current workflowID. - """ + """Updates workflow status.""" headers = self.request_headers self.update_status_request(headers) diff --git a/src/dags/libs/upload_file.py b/src/dags/libs/upload_file.py deleted file mode 100644 index 4014e9722b1299f397af786f6ec61397b5ced2e1..0000000000000000000000000000000000000000 --- a/src/dags/libs/upload_file.py +++ /dev/null @@ -1,178 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -import io -import json -import logging -from abc import ABC, abstractmethod -from typing import Tuple, TypeVar -from urllib.parse import urlparse - -import requests -import tenacity -from google.cloud import storage -from libs.context import Context -from libs.exceptions import GCSObjectURIError, FileSourceError -from libs.mixins import HeadersMixin -from libs.refresh_token import TokenRefresher, refresh_token - -logger = logging.getLogger() - -FileLikeObject = TypeVar("FileLikeObject", io.IOBase, io.RawIOBase, io.BytesIO) - -RETRY_SETTINGS = { - "stop": tenacity.stop_after_attempt(3), - "wait": tenacity.wait_fixed(2), -} - - -class FileUploader(HeadersMixin, ABC): - """ - File uploader to copy file from PreLoadPath into FileSource on OSDU platform. - """ - - def __init__(self, file_service: str, token_refresher: TokenRefresher, context: Context): - super().__init__(context) - self.file_service = file_service - self.token_refresher = token_refresher - - @abstractmethod - def get_file_from_preload_path(self, preload_path: str) -> FileLikeObject: - """ - Return file-like object containing raw content of a file - in preload path. - """ - - @tenacity.retry(**RETRY_SETTINGS) - @refresh_token() - def _send_post_request(self, headers: dict, url: str, request_body: str) -> requests.Response: - response = requests.post(url, request_body, headers=headers) - return response - - @tenacity.retry(**RETRY_SETTINGS) - def _get_signed_url_request(self, headers: dict) -> Tuple[str, str]: - """ - Get fileID and SignedURL using File Service. - """ - logger.debug("Getting signed url.") - request_body = json.dumps({}) # '/getLocation' method requires empty json. - response = self._send_post_request(headers, f"{self.file_service}/getLocation", - request_body).json() - logger.debug("Signed url got.") - logger.debug(response) - return response["FileID"], response["Location"]["SignedURL"] - - @tenacity.retry(**RETRY_SETTINGS) - def _upload_file_request(self, headers: dict, signed_url: str, buffer: FileLikeObject): - """ - Upload file via File service using signed_url. - """ - logger.debug("Uploading file.") - buffer.seek(0) - requests.put(signed_url, buffer.read(), headers=headers) - logger.debug("File uploaded.") - - @tenacity.retry(**RETRY_SETTINGS) - def _get_file_location_request(self, headers: dict, file_id: str) -> str: - """ - Get file location using File Service. - """ - logger.debug("Getting file location.") - request_body = json.dumps({"FileID": file_id}) - response = self._send_post_request(headers, f"{self.file_service}/getFileLocation", - request_body).json() - logger.debug("File location got.") - return response["Location"] - - def upload_file(self, preload_file_path: str) -> str: - """ - Copy file from Landing zone(preload_file_path) onto OSDU platform using File service. - Return file_location. - """ - buffer = self.get_file_from_preload_path(preload_file_path) - file_id, signed_url = self._get_signed_url_request(self.request_headers) - self._upload_file_request(self.request_headers, signed_url, buffer) - file_location = self._get_file_location_request(self.request_headers, file_id) - return file_location - - -class GCSFileUploader(FileUploader): - - def __init__( - self, - file_service: str, - token_refresher: TokenRefresher, - context: Context, - ): - super().__init__(file_service, token_refresher, context) - - @staticmethod - def _parse_object_uri(file_path: str) -> Tuple[str, str]: - """ - Parse GCS Object uri. - Return bucket and blob names. - """ - parsed_path = urlparse(file_path) - if parsed_path.scheme == "gs": - bucket_name = parsed_path.netloc - source_blob_name = parsed_path.path[1:] # delete the first slash - - if bucket_name and source_blob_name: - return bucket_name, source_blob_name - - raise GCSObjectURIError - - @tenacity.retry(**RETRY_SETTINGS) - def get_file_from_bucket( - self, - bucket_name: str, - source_blob_name: str - ) -> Tuple[io.BytesIO, str]: - storage_client = storage.Client() - bucket = storage_client.bucket(bucket_name) - blob = bucket.get_blob(source_blob_name) - - does_exist = blob.exists() - if not does_exist: - raise FileSourceError("File doesn't exist in preloadPath " - f"'gs://{bucket_name}/{source_blob_name}'") - - file = io.BytesIO() - blob.download_to_file(file) - logger.debug("File got from landing zone") - return file, blob.content_type - - def get_file_from_preload_path(self, preload_file_path: str) -> Tuple[io.BytesIO, str]: - bucket_name, blob_name = self._parse_object_uri(preload_file_path) - buffer, content_type = self.get_file_from_bucket(bucket_name, blob_name) - return buffer, content_type - - def upload_file(self, preload_file_path: str) -> str: - """ - Copy file from Landing zone(preload_file_path) onto OSDU platform using File service. - Get Content-Type of this file, refresh Content-Type with this value in headers - while this file is being uploaded onto OSDU platform. - Return file_location. - """ - buffer, content_type = self.get_file_from_preload_path(preload_file_path) - file_id, signed_url = self._get_signed_url_request(self.request_headers) - - headers = self.request_headers - headers["Content-Type"] = content_type - self._upload_file_request(headers, signed_url, buffer) - - file_location = self._get_file_location_request(self.request_headers, file_id) - return file_location diff --git a/src/dags/libs/validate_schema.py b/src/dags/libs/validate_schema.py index 4ae82b47f446443fdae768d886bee6b756cb9844..b72647baccc912e540fd85f48a556497b6c13b61 100644 --- a/src/dags/libs/validate_schema.py +++ b/src/dags/libs/validate_schema.py @@ -13,14 +13,19 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Provides SchemaValidator.""" + import copy +import json import logging +from typing import Union, Any, List import jsonschema import requests import tenacity +from jsonschema import exceptions from libs.context import Context -from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError +from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError from libs.mixins import HeadersMixin from libs.refresh_token import TokenRefresher, refresh_token @@ -31,17 +36,30 @@ TIMEOUT = 1 class OSDURefResolver(jsonschema.RefResolver): + """Extends base jsonschema resolver for OSDU.""" def __init__(self, schema_service: str, *args, **kwargs): + """Implements the schema validatoe + + :param schema_service: The base url for schema service + :type schema_service: str + """ super(OSDURefResolver, self).__init__(*args, **kwargs) self.schema_service = schema_service def resolve_fragment(self, document: dict, fragment: str) -> dict: - """ - Extend base resolve_fragment method. If a nested schema has 'definitions' field and there - is a schema under this 'definitions', jsonschema attempts to use the id field of this - double-nested schema as URI to get this schema later. So it has sense to replace this id + """Extend base resolve_fragment method. If a nested schema has + 'definitions' field and there is a schema under this 'definitions', + jsonschema attempts to use the id field of this double-nested schema + as URI to get this schema later. So it has sense to replace this id with a correct one. + + :param document: The schema document + :type document: dict + :param fragment: schema fragment + :type fragment: str + :return: The updated schema document + :rtype: dict """ document = super().resolve_fragment(document, fragment) fragment_parts = fragment.split("/") # /definitions/<OsduID> -> [..., <OsduID>] @@ -57,13 +75,22 @@ class SchemaValidator(HeadersMixin): def __init__( self, schema_service: str, - dagrun_conf: dict, token_refresher: TokenRefresher, context: Context ): + """Init SchemaValidator. + + :param schema_service: The base OSDU Schema service url + :type schema_service: str + :param dagrun_conf: The airflow dagrun.conf + :type dagrun_conf: dict + :param token_refresher: An instance of token refresher + :type token_refresher: TokenRefresher + :param context: The tenant context + :type context: Context + """ super().__init__(context) self.schema_service = schema_service - self.data_object = copy.deepcopy(dagrun_conf) self.context = context self.token_refresher = token_refresher self.resolver_handlers = { @@ -79,15 +106,34 @@ class SchemaValidator(HeadersMixin): ) @refresh_token() def _get_schema_from_schema_service(self, headers: dict, uri: str) -> requests.Response: - """ - Request to Schema service to retrieve schema. - """ + """Send request to Schema service to retrieve schema.""" response = requests.get(uri, headers=headers, timeout=60) return response - def get_schema_request(self, uri: str) -> dict: + def __delete_refs(self, schema_part: Union[dict, list]): """ - Get Schema from Schema service. Change $id field to url of getting schema. + Recursively clear a schema's object parts containing "$ref". + 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) + + def get_schema_request(self, uri: str) -> dict: + """Get schema from Schema service. Change $id field to url. + + :param uri: The URI to fetch the schema + :type uri: str + :return: The Schema service response + :rtype: dict """ if uri.startswith("osdu") or uri.startswith(self.context.data_partition_id): uri = f"{self.schema_service}/{uri}" @@ -96,6 +142,14 @@ class SchemaValidator(HeadersMixin): return response def get_schema(self, kind: str) -> dict: + """Fetch schema from Schema service. + + :param kind: The kind of the scheema to fetch + :type kind: str + :raises e: Generic exception + :return: Schema server response + :rtype: dict + """ manifest_schema_uri = f"{self.schema_service}/{kind}" try: response = self.get_schema_request(manifest_schema_uri) @@ -104,43 +158,126 @@ class SchemaValidator(HeadersMixin): raise e return response - def _validate_schema(self, manifest: dict, schema: dict = None): + def _delete_id_pattern(self, schema: dict) -> dict: + if schema.get("properties") and schema["properties"].get("id"): + schema["properties"]["id"].pop("pattern", None) + return schema + + def _validate_entity(self, entity: dict, schema: dict = None): """ - Validate schema. If argument schema is not defined, then use schema service to retrieve - corresponding schema. + Validate the 'data' field of any entity against a schema got by entity's kind. """ if not schema: - schema = self.get_schema(manifest["kind"]) - if schema["properties"].get("id"): - schema["properties"]["id"].pop("pattern", None) - logger.debug(f"Validating kind {manifest['kind']}") - resolver = OSDURefResolver(schema_service=self.schema_service, - base_uri=schema.get("$id", ""), referrer=schema, - handlers=self.resolver_handlers, cache_remote=True) - validator = jsonschema.Draft7Validator(schema=schema, resolver=resolver) - validator.validate(manifest) + schema = self.get_schema(entity["kind"]) + schema = self._delete_id_pattern(schema) + data = entity["data"] + try: + self._validate_against_schema(schema, data) + logger.debug(f"Record successfully validated") + return True + except exceptions.ValidationError as exc: + logger.error("Schema validation error. Data field.") + logger.error(f"Manifest kind: {entity['kind']}") + logger.error(f"Error: {exc}") + return False - def validate_work_product(self, work_product: dict): + def _validate_work_product(self, work_product: dict): """ Validate WP manifest. Raise error if manifest is not valid. """ for key, value in work_product.items(): if key != "WorkProduct": for component in value: - self._validate_schema(component) + self._validate_entity(component) else: - self._validate_schema(value) + self._validate_entity(value) + + def _validate_against_schema(self, schema: dict, data: Any): + """ + Validate any data against schema. + :param schema: + :param data: + :return: + """ + resolver = OSDURefResolver( + schema_service=self.schema_service, + base_uri=schema.get("$id", ""), + referrer=schema, + handlers=self.resolver_handlers, + cache_remote=True + ) + validator = jsonschema.Draft7Validator(schema=schema, resolver=resolver) + validator.validate(data) + + def _validate_data_group(self, entities: list): + """ + Validate each entity from a list of entities. + :param entities: + :return: + """ + if isinstance(entities, list): + for entity in entities: + self._validate_entity(entity) + + def _validate_whole_manifest(self, manifest: dict): + """ + Validate any manifest in general. + Also at this step verify that MasterData, ReferenceData, WorkProduct, WorkProductComponents, + Files entities correspond their generic schemas, because references to those schemas are in + a Manifest schema. + """ + schema = self.get_schema(manifest["kind"]) + logger.debug(f"Validating kind {manifest['kind']}") + self._validate_against_schema(schema, manifest) + + def validate_common_schema(self, manifest: dict) -> dict: + """ + This is a preliminary validation of a manifest that verifies that a manifest corresponds + the OSDU schemes at the highest level. + This validation skips verifying each concrete entity by removing references to their schemas. + :param manifest: + :return: Manifest schema + """ + if "manifest" not in manifest: + raise EmptyManifestError + schema = self.get_schema(manifest["manifest"]["kind"]) + schema_without_refs = copy.deepcopy(schema) + if schema_without_refs.get("properties"): + self.__delete_refs(schema_without_refs["properties"]) + else: + self.__delete_refs(schema_without_refs) + logger.debug("Schema without refs") + logger.debug(f"{schema_without_refs}") + self._validate_against_schema(schema, manifest) + return schema + + def _validate_against_generic_schema(self, schema: str, entity: Any) -> bool: + try: + self._validate_against_schema(schema, entity) + logger.debug(f"Record successfully validated against generic schema.") + return True + except exceptions.ValidationError as exc: + logger.error("Schema validation error.") + logger.error(f"Manifest kind: {schema['kind']}") + logger.error(f"Manifest: {entity}") + logger.error(f"Error: {exc}") + return False - def validate_manifest(self): + def validate_manifest(self, manifest_records: List[dict]) -> List[dict]: """ Validate manifest. Raise error if manifest is not valid. """ - if "manifest" not in self.data_object: + validated_records = [] + if not manifest_records: raise EmptyManifestError - for manifest in self.data_object["manifest"]: + for manifest_record in manifest_records: + manifest = manifest_record.get("entity") if isinstance(manifest, dict) and manifest.get("kind"): - self._validate_schema(manifest) - elif manifest.get("WorkProductComponents"): - self.validate_work_product(manifest) + generic_schema = self.get_schema(manifest_record.get("schema")) + validation_result = self._validate_against_generic_schema(generic_schema, manifest) \ + and self._validate_entity(manifest) + if validation_result: + validated_records.append(manifest_record) else: - raise NotOSDUShemaFormatError(f"Not valid schema {manifest}") + raise NotOSDUSchemaFormatError(f"Not valid schema {manifest}") + return validated_records diff --git a/src/dags/osdu-ingest-r2.py b/src/dags/osdu-ingest-r2.py index 37e1de1c8b5f49ea026000ffb8ab8b0bc0eb0886..f59aab893c9833324b2c431bcfffe3780df2c317 100644 --- a/src/dags/osdu-ingest-r2.py +++ b/src/dags/osdu-ingest-r2.py @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -"""DAG for R2 ingestion""" +"""DAG for R2 ingestion.""" from datetime import timedelta @@ -63,4 +63,4 @@ search_record_ids_op = SearchRecordIdOperator( ) update_status_running_op >> process_manifest_op >> \ -search_record_ids_op >> update_status_finished_op # pylint: disable=pointless-statement +search_record_ids_op >> update_status_finished_op # pylint: disable=pointless-statement diff --git a/src/dags/osdu-ingest-r3.py b/src/dags/osdu-ingest-r3.py index 29b61fe8d19b2ad7b2289afcc30e8af56d439829..2354ed84d47be03afea7f6c4e971a26b17aa364f 100644 --- a/src/dags/osdu-ingest-r3.py +++ b/src/dags/osdu-ingest-r3.py @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -"""DAG for R3 ingestion""" +"""DAG for R3 ingestion.""" from datetime import timedelta diff --git a/src/dags/well-log-ingest.py b/src/dags/well-log-ingest.py index 24d949b6054d0c449a6d7431f103d7d79a9970db..bca78dd7ddd1dbad57d63d84bf7844eab99db845 100644 --- a/src/dags/well-log-ingest.py +++ b/src/dags/well-log-ingest.py @@ -12,6 +12,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""WellLog DAG.""" + from datetime import timedelta import airflow diff --git a/src/plugins/hooks/http_hooks.py b/src/plugins/hooks/http_hooks.py index 34fe9b793e359c3aefde73f380bebc7c57a9140a..cc747c167aca5c799008038ed4c9068f63840738 100644 --- a/src/plugins/hooks/http_hooks.py +++ b/src/plugins/hooks/http_hooks.py @@ -13,6 +13,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Http Hooks.""" + from airflow.hooks.http_hook import HttpHook workflow_hook = HttpHook(http_conn_id='workflow', method="POST") diff --git a/src/plugins/operators/process_manifest_r2.py b/src/plugins/operators/process_manifest_r2.py index d77987c48e85f027d9ce8801658579ad5d1f374e..e85ad3aeea2f41da20b7eaac740fef064a3ad33c 100644 --- a/src/plugins/operators/process_manifest_r2.py +++ b/src/plugins/operators/process_manifest_r2.py @@ -13,6 +13,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""R2 Process Manifest operator.""" + import configparser import enum import json @@ -73,6 +75,11 @@ def dataload(**kwargs): def create_headers(conf_payload): + """Create header. + + :param conf_payload: config payload + :return: headers + """ partition_id = conf_payload["data-partition-id"] app_key = conf_payload["AppKey"] headers = { @@ -84,8 +91,7 @@ def create_headers(conf_payload): def generate_id(type_id): - """ - Generate resource ID + """Generate resource ID. :param type_id: resource type ID :return: resource ID @@ -94,8 +100,7 @@ def generate_id(type_id): def determine_data_type(raw_resource_type_id): - """ - Determine resource type ID + """Determine resource type ID. :param raw_resource_type_id: raw resource type ID from manifest file :return: short resource type ID @@ -106,6 +111,12 @@ def determine_data_type(raw_resource_type_id): # TODO: add comments to functions that implement actions in this function def process_file_items(loaded_conf, conf_payload) -> Tuple[list, list]: + """Process files items. + + :param loaded_conf: loaded configuration + :param conf_payload: configuration payload + :return: list of file records and list of their ids + """ file_ids = [] file_list = [] data_object = loaded_conf.get("data_object") @@ -124,6 +135,14 @@ def process_file_items(loaded_conf, conf_payload) -> Tuple[list, list]: def process_wpc_items(loaded_conf, product_type, file_ids, conf_payload): + """Process WorkProductComponents items. + + :param loaded_conf: loaded configuration + :param product_type: product type + :param file_ids: list of file ids + :param conf_payload: configuration payload + :return: list of workproductcomponents records and list of their ids + """ wpc_ids = [] wpc_list = [] data_object = loaded_conf.get("data_object") @@ -143,6 +162,14 @@ def process_wpc_items(loaded_conf, product_type, file_ids, conf_payload): def process_wp_item(loaded_conf, product_type, wpc_ids, conf_payload) -> list: + """Process WorkProduct item. + + :param loaded_conf: loaded configuration + :param product_type: product type + :param wpc_ids: work product component ids + :param conf_payload: configuration payload + :return: work product record + """ data_object = loaded_conf.get("data_object") acl = loaded_conf.get("acl") legal_tag = loaded_conf.get("legal_tag") @@ -160,12 +187,22 @@ def process_wp_item(loaded_conf, product_type, wpc_ids, conf_payload) -> list: def validate_file_type(file_type, data_object): + """Validate file type. + + :param file_type: file type + :param data_object: file record + """ if not file_type: logger.error(f"Error with file {data_object}. Type could not be specified.") sys.exit(2) def validate_file(loaded_conf) -> Tuple[FileType, str]: + """Validate file. + + :param loaded_conf: loaded configuration + :return: file type and produc type + """ data_object = loaded_conf.get("data_object") if not data_object: logger.error(f"Error with file {data_object}. It is empty.") @@ -191,6 +228,12 @@ def validate_file(loaded_conf) -> Tuple[FileType, str]: def create_kind(data_kind, conf_payload): + """Create kind. + + :param data_kind: data kind + :param conf_payload: configuration payload + :return: kind + """ partition_id = conf_payload.get("data-partition-id", DEFAULT_TENANT) source = conf_payload.get("authority", DEFAULT_SOURCE) version = conf_payload.get("kind_version", DEFAULT_VERSION) @@ -200,8 +243,7 @@ def create_kind(data_kind, conf_payload): def populate_request_body(data, acl, legal_tag, data_type, conf_payload): - """ - Populate request body according API specification + """Populate request body according API specification :param data: item data from manifest files :param data_type: resource type ID @@ -226,8 +268,7 @@ def populate_request_body(data, acl, legal_tag, data_type, conf_payload): def separate_type_data(request_data): - """ - Separate the list of tuples into Data Type Counter and data list + """Separate the list of tuples into Data Type Counter and data list :param request_data: tuple of data and types :type request_data: tuple(list, str) @@ -267,9 +308,7 @@ def create_workproduct_request_data(loaded_conf: dict, product_type: str, wp, wp ) @refresh_token(AirflowTokenRefresher()) def send_request(headers, request_data): - """ - Send request to records storage API - """ + """Send request to records storage API.""" logger.error(f"Header {str(headers)}") # loop for implementing retries send process @@ -313,6 +352,7 @@ def send_request(headers, request_data): def process_manifest(**kwargs): + """Process manifest.""" loaded_conf, conf_payload = dataload(**kwargs) file_type, product_type = validate_file(loaded_conf) if file_type is FileType.MANIFEST: @@ -333,6 +373,7 @@ def process_manifest(**kwargs): class ProcessManifestOperatorR2(BaseOperator): + """R2 Manifest Operator.""" ui_color = '#dad5ff' ui_fgcolor = '#000000' diff --git a/src/plugins/operators/process_manifest_r3.py b/src/plugins/operators/process_manifest_r3.py index 694d9e4b93b277c984647c413dfa24f837965439..df16b8c6014d92323686874e512704a533729f9e 100644 --- a/src/plugins/operators/process_manifest_r3.py +++ b/src/plugins/operators/process_manifest_r3.py @@ -13,56 +13,71 @@ # See the License for the specific language governing permissions and # limitations under the License. -import logging +"""R2 Process Manifest operator.""" +import logging from airflow.utils import apply_defaults from airflow.models import BaseOperator, Variable from libs.context import Context from libs.source_file_check import GCSSourceFileChecker -from libs.upload_file import GCSFileUploader +from libs.handle_file import GCSFileHandler 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 logger = logging.getLogger() -RETRIES = 3 -TIMEOUT = 1 - class ProcessManifestOperatorR3(BaseOperator): + """Operator to process manifest R3.""" + ui_color = '#dad5ff' ui_fgcolor = '#000000' @apply_defaults - def __init__(self, *args, **kwargs) -> None: + def __init__(self, *args, **kwargs): + """Init base operator and obtain base urls from Airflow Variables.""" super().__init__(*args, **kwargs) 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') 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 + + :param context: Airflow context + :type context: dict + """ payload_context = Context.populate(context["dag_run"].conf) token_refresher = AirflowTokenRefresher() - file_uploader = GCSFileUploader(self.file_service_url, token_refresher, - payload_context) + file_handler = GCSFileHandler(self.file_service_url, token_refresher, payload_context) source_file_checker = GCSSourceFileChecker() - validator = SchemaValidator( self.schema_service_url, - context["dag_run"].conf, token_refresher, payload_context ) + manifest_schema = validator.validate_common_schema(context["dag_run"].conf) + traversal = ManifestTraversal(context["dag_run"].conf, 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_processor = ManifestProcessor( self.storage_url, - context["dag_run"].conf, - file_uploader, + valid_manifest_entities, + file_handler, source_file_checker, token_refresher, payload_context, ) - - validator.validate_manifest() record_ids = manifest_processor.process_manifest() context["ti"].xcom_push(key="record_ids", value=record_ids) diff --git a/src/plugins/operators/search_record_id.py b/src/plugins/operators/search_record_id.py index d122dc6507e74320adfffeb928c486e481030020..d457c252258b428747ba499ac385b0f47406ec79 100644 --- a/src/plugins/operators/search_record_id.py +++ b/src/plugins/operators/search_record_id.py @@ -24,8 +24,7 @@ logger = logging.getLogger() class SearchRecordIdOperator(BaseOperator): - """ - Operator to search files in SearchService by record ids. + """Operator to search files in SearchService by record ids. Expects "record_ids" field in xcom. """ ui_color = '#10ECAA' @@ -37,8 +36,12 @@ class SearchRecordIdOperator(BaseOperator): def execute(self, context: dict): """Execute update workflow status. - If status assumed to be FINISHED then we check whether proceed files are searchable or not. - If they are then update status FINISHED else FAILED + If status assumed to be FINISHED then we check whether proceed files + are searchable or not. + If they are then update status FINISHED else FAILED + + :param context: Airflow dagrun context + :type context: dict """ payload_context = Context.populate(context["dag_run"].conf) record_ids = context["ti"].xcom_pull(key="record_ids", ) diff --git a/src/plugins/operators/update_status.py b/src/plugins/operators/update_status.py index c2b70bca91119b6901d15a6e09b223aa3e92f2f0..43c01269d290c13896b08eff8370d3676703db4e 100644 --- a/src/plugins/operators/update_status.py +++ b/src/plugins/operators/update_status.py @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +"""Update Status operator.""" import copy import enum @@ -29,6 +30,7 @@ logger = logging.getLogger() class UpdateStatusOperator(BaseOperator): + """Operator to update status.""" ui_color = '#10ECAA' ui_fgcolor = '#000000' @@ -37,10 +39,14 @@ class UpdateStatusOperator(BaseOperator): SUCCESS = "finished" FAILED = "failed" - def get_previous_ti_statuses(self, context: dict): - """ - Get status of previous tasks' executions. + def get_previous_ti_statuses(self, context: dict) -> enum.Enum: + """Get status of previous tasks' executions. Return corresponding enum value. + + :param context: Airflow context + :type context: dict + :return: Previous status + :rtype: enum.Enum """ dagrun = context['ti'].get_dagrun() failed_ti = dagrun.get_task_instances(state='failed') @@ -59,8 +65,13 @@ class UpdateStatusOperator(BaseOperator): def execute(self, context: dict): """Execute update workflow status. - If status assumed to be FINISHED then we check whether proceed files are searchable or not. - If they are then update status FINISHED else FAILED + If status assumed to be FINISHED then we check whether records + are searchable or not. + If they are then update status FINISHED else FAILED + + :param context: Airflow context + :type context: dict + :raises PipelineFailedError: If any of the status is failed """ conf = copy.deepcopy(context["dag_run"].conf) logger.debug(f"Got conf {conf}.") @@ -81,5 +92,3 @@ class UpdateStatusOperator(BaseOperator): status_updater.update_workflow_status() if self.status is self.prev_ti_state.FAILED: raise PipelineFailedError("Dag failed") - - diff --git a/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json b/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json index 62b46151c77ce0fc7a8b851f824460b644c918fc..88b88156abb890ed9fee9154e58ac9801922780c 100644 --- a/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json +++ b/tests/plugin-unit-tests/data/invalid/NotOsduFormat.json @@ -7,10 +7,6 @@ }, "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - { - - } - ], + "manifest": {"test": "test"}, "WorkflowID": "foo" } diff --git a/tests/plugin-unit-tests/data/invalid/TraversalEmptyManifest.json b/tests/plugin-unit-tests/data/invalid/TraversalEmptyManifest.json new file mode 100644 index 0000000000000000000000000000000000000000..0d4f101c7a37a4c875e6999bee1a287fdb733380 --- /dev/null +++ b/tests/plugin-unit-tests/data/invalid/TraversalEmptyManifest.json @@ -0,0 +1,2 @@ +[ +] diff --git a/tests/plugin-unit-tests/data/invalid/TraversalNotOSDUFormatManifest.json b/tests/plugin-unit-tests/data/invalid/TraversalNotOSDUFormatManifest.json new file mode 100644 index 0000000000000000000000000000000000000000..5c1b8302fd050961756fa635cae81195d8a8ee60 --- /dev/null +++ b/tests/plugin-unit-tests/data/invalid/TraversalNotOSDUFormatManifest.json @@ -0,0 +1,8 @@ +[ + { + "entity": {"test": "test"}, + "group_type": "", + "schema": "GenericMasterData" + } +] + diff --git a/tests/plugin-unit-tests/data/manifests/Manifest.1.0.0.json b/tests/plugin-unit-tests/data/manifests/Manifest.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..3a1833c6939b9cacca07511719af26194f2588cd --- /dev/null +++ b/tests/plugin-unit-tests/data/manifests/Manifest.1.0.0.json @@ -0,0 +1,559 @@ +{ + "kind": "osdu:wks:Manifest:1.0.0", + "ReferenceData": [ + { + "id": "namespace:reference-data--GenericReferenceData:63ca0ed3-d6fb-53f0-8549-0916ef144266", + "kind": "osdu:wks:reference-data--GenericReferenceData:1.0.0", + "version": 1562066009929332, + "acl": { + "owners": [ + "someone@company.com" + ], + "viewers": [ + "someone@company.com" + ] + }, + "legal": { + "legaltags": [ + "Example legaltags" + ], + "otherRelevantDataCountries": [ + "US" + ], + "status": "compliant" + }, + "resourceHomeRegionID": "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:", + "resourceHostRegionIDs": [ + "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:" + ], + "createTime": "2020-12-16T11:46:20.163Z", + "createUser": "some-user@some-company-cloud.com", + "modifyTime": "2020-12-16T11:52:24.477Z", + "modifyUser": "some-user@some-company-cloud.com", + "resourceCurationStatus": "namespace:reference-data--ResourceCurationStatus:CREATED:", + "resourceLifecycleStatus": "namespace:reference-data--ResourceLifecycleStatus:LOADING:", + "resourceSecurityClassification": "namespace:reference-data--ResourceSecurityClassification:SomeUniqueResourceSecurityClassificationID:", + "ancestry": { + "parents": [] + }, + "meta": [ + { + "kind": "CRS", + "name": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "persistableReference": "{\"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]]\"}", + "coordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:", + "propertyNames": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + ], + "source": "Example Data Source", + "existenceKind": "namespace:reference-data--ExistenceKind:Prototype:", + "data": {} + } + ], + "MasterData": [ + { + "id": "namespace:master-data--GenericMasterData:9ca8054c-bce6-5a3a-b51d-f216fb1085a5", + "kind": "osdu:wks:master-data--GenericMasterData:1.0.0", + "version": 1562066009929332, + "acl": { + "owners": [ + "someone@company.com" + ], + "viewers": [ + "someone@company.com" + ] + }, + "legal": { + "legaltags": [ + "Example legaltags" + ], + "otherRelevantDataCountries": [ + "US" + ], + "status": "compliant" + }, + "resourceHomeRegionID": "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:", + "resourceHostRegionIDs": [ + "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:" + ], + "createTime": "2020-12-16T11:46:20.163Z", + "createUser": "some-user@some-company-cloud.com", + "modifyTime": "2020-12-16T11:52:24.477Z", + "modifyUser": "some-user@some-company-cloud.com", + "resourceCurationStatus": "namespace:reference-data--ResourceCurationStatus:CREATED:", + "resourceLifecycleStatus": "namespace:reference-data--ResourceLifecycleStatus:LOADING:", + "resourceSecurityClassification": "namespace:reference-data--ResourceSecurityClassification:SomeUniqueResourceSecurityClassificationID:", + "ancestry": { + "parents": [] + }, + "meta": [ + { + "kind": "CRS", + "name": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "persistableReference": "{\"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]]\"}", + "coordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:", + "propertyNames": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + ], + "source": "Example Data Source", + "existenceKind": "namespace:reference-data--ExistenceKind:Prototype:", + "data": {} + } + ], + "Data": { + "WorkProduct": { + "id": "surrogate-key:wp-id-1", + "kind": "osdu:wks:work-product--GenericWorkProduct:1.0.0", + "version": 1562066009929332, + "acl": { + "owners": [ + "someone@company.com" + ], + "viewers": [ + "someone@company.com" + ] + }, + "legal": { + "legaltags": [ + "Example legaltags" + ], + "otherRelevantDataCountries": [ + "US" + ], + "status": "compliant" + }, + "resourceHomeRegionID": "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:", + "resourceHostRegionIDs": [ + "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:" + ], + "createTime": "2020-12-16T11:46:20.163Z", + "createUser": "some-user@some-company-cloud.com", + "modifyTime": "2020-12-16T11:52:24.477Z", + "modifyUser": "some-user@some-company-cloud.com", + "resourceCurationStatus": "namespace:reference-data--ResourceCurationStatus:CREATED:", + "resourceLifecycleStatus": "namespace:reference-data--ResourceLifecycleStatus:LOADING:", + "resourceSecurityClassification": "namespace:reference-data--ResourceSecurityClassification:SomeUniqueResourceSecurityClassificationID:", + "ancestry": { + "parents": [] + }, + "meta": [ + { + "kind": "CRS", + "name": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "persistableReference": "{\"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]]\"}", + "coordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:", + "propertyNames": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + ], + "source": "Example Data Source", + "existenceKind": "namespace:reference-data--ExistenceKind:Prototype:", + "data": { + "Components": [ + "surrogate-key:wpc-id-1" + ], + "IsExtendedLoad": true, + "IsDiscoverable": true, + "Name": "Example Name", + "Description": "Example Description", + "CreationDateTime": "2020-02-13T09:13:15.55Z", + "Tags": [ + "Example Tags" + ], + "SpatialPoint": { + "SpatialLocationCoordinatesDate": "2020-02-13T09:13:15.55Z", + "QuantitativeAccuracyBandID": "namespace:reference-data--QuantitativeAccuracyBand:SomeUniqueQuantitativeAccuracyBandID:", + "QualitativeSpatialAccuracyTypeID": "namespace:reference-data--QualitativeSpatialAccuracyType:SomeUniqueQualitativeSpatialAccuracyTypeID:", + "CoordinateQualityCheckPerformedBy": "Example CoordinateQualityCheckPerformedBy", + "CoordinateQualityCheckDateTime": "2020-02-13T09:13:15.55Z", + "CoordinateQualityCheckRemarks": [ + "Example CoordinateQualityCheckRemarks" + ], + "AsIngestedCoordinates": { + "type": "AnyCrsFeatureCollection", + "CoordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:BoundCRS.SLB.32021.15851:", + "VerticalCoordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:VerticalCRS.EPSG.5773:", + "persistableReferenceCRS": "{\"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\"}", + "persistableReferenceVerticalCRS": "{\"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]]\"}", + "persistableReferenceUnitZ": "{\"scaleOffset\":{\"scale\":1.0,\"offset\":0.0},\"symbol\":\"m\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}", + "features": [ + { + "type": "AnyCrsFeature", + "properties": {}, + "geometry": { + "type": "AnyCrsPoint", + "coordinates": [ + 12345.6, + 12345.6 + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + } + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "properties": {}, + "geometry": { + "type": "Point", + "coordinates": [ + 12345.6, + 12345.6 + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + } + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "OperationsApplied": [ + "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" + ], + "SpatialParameterTypeID": "namespace:reference-data--SpatialParameterType:Outline:", + "SpatialGeometryTypeID": "namespace:reference-data--SpatialGeometryType:Point:" + }, + "SpatialArea": { + "SpatialLocationCoordinatesDate": "2020-02-13T09:13:15.55Z", + "QuantitativeAccuracyBandID": "namespace:reference-data--QuantitativeAccuracyBand:SomeUniqueQuantitativeAccuracyBandID:", + "QualitativeSpatialAccuracyTypeID": "namespace:reference-data--QualitativeSpatialAccuracyType:SomeUniqueQualitativeSpatialAccuracyTypeID:", + "CoordinateQualityCheckPerformedBy": "Example CoordinateQualityCheckPerformedBy", + "CoordinateQualityCheckDateTime": "2020-02-13T09:13:15.55Z", + "CoordinateQualityCheckRemarks": [ + "Example CoordinateQualityCheckRemarks" + ], + "AsIngestedCoordinates": { + "type": "AnyCrsFeatureCollection", + "CoordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:BoundCRS.SLB.32021.15851:", + "VerticalCoordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:VerticalCRS.EPSG.5773:", + "persistableReferenceCRS": "{\"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\"}", + "persistableReferenceVerticalCRS": "{\"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]]\"}", + "persistableReferenceUnitZ": "{\"scaleOffset\":{\"scale\":1.0,\"offset\":0.0},\"symbol\":\"m\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}", + "features": [ + { + "type": "AnyCrsFeature", + "properties": {}, + "geometry": { + "type": "AnyCrsPoint", + "coordinates": [ + 12345.6, + 12345.6 + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + } + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "properties": {}, + "geometry": { + "type": "Point", + "coordinates": [ + 12345.6, + 12345.6 + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + } + ], + "bbox": [ + 12345.6, + 12345.6, + 12345.6, + 12345.6 + ] + }, + "OperationsApplied": [ + "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" + ], + "SpatialParameterTypeID": "namespace:reference-data--SpatialParameterType:Outline:", + "SpatialGeometryTypeID": "namespace:reference-data--SpatialGeometryType:Point:" + }, + "SubmitterName": "Example SubmitterName", + "BusinessActivities": [ + "Example BusinessActivities" + ], + "AuthorIDs": [ + "Example AuthorIDs" + ], + "LineageAssertions": [ + { + "ID": "namespace:any-group-type--AnyIndividualType:SomeUniqueAnyIndividualTypeID:", + "LineageRelationshipType": "namespace:reference-data--LineageRelationshipType:Direct:" + } + ], + "Annotations": [ + "Example Annotations" + ] + } + }, + "WorkProductComponents": [ + { + "id": "surrogate-key:wpc-id-1", + "kind": "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0", + "version": 1562066009929332, + "acl": { + "owners": [ + "someone@company.com" + ], + "viewers": [ + "someone@company.com" + ] + }, + "legal": { + "legaltags": [ + "Example legaltags" + ], + "otherRelevantDataCountries": [ + "US" + ], + "status": "compliant" + }, + "resourceHomeRegionID": "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:", + "resourceHostRegionIDs": [ + "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:" + ], + "createTime": "2020-12-16T11:46:20.163Z", + "createUser": "some-user@some-company-cloud.com", + "modifyTime": "2020-12-16T11:52:24.477Z", + "modifyUser": "some-user@some-company-cloud.com", + "resourceCurationStatus": "namespace:reference-data--ResourceCurationStatus:CREATED:", + "resourceLifecycleStatus": "namespace:reference-data--ResourceLifecycleStatus:LOADING:", + "resourceSecurityClassification": "namespace:reference-data--ResourceSecurityClassification:SomeUniqueResourceSecurityClassificationID:", + "ancestry": { + "parents": [] + }, + "meta": [ + { + "kind": "CRS", + "name": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "persistableReference": "{\"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]]\"}", + "coordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:", + "propertyNames": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + ], + "source": "Example Data Source", + "existenceKind": "namespace:reference-data--ExistenceKind:Prototype:", + "data": { + "Datasets": [ + "surrogate-key:.+" + ], + "Artefacts": [ + { + "RoleID": "namespace:reference-data--ArtefactRole:SomeUniqueArtefactRoleID:", + "ResourceKind": "namespace:source_name:group_type/IndividualType:0.0.0", + "ResourceID": "surrogate-key:file-2" + } + ], + "IsExtendedLoad": true, + "IsDiscoverable": true, + "DatasetIDs": [ + "surrogate-key:file-1" + ] + } + } + ], + "Datasets": [ + { + "id": "surrogate-key:file-1", + "kind": "osdu:wks:dataset--GenericDataset:1.0.0", + "version": 1562066009929332, + "acl": { + "owners": [ + "someone@company.com" + ], + "viewers": [ + "someone@company.com" + ] + }, + "legal": { + "legaltags": [ + "Example legaltags" + ], + "otherRelevantDataCountries": [ + "US" + ], + "status": "compliant" + }, + "resourceHomeRegionID": "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:", + "resourceHostRegionIDs": [ + "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:" + ], + "createTime": "2020-12-16T11:46:20.163Z", + "createUser": "some-user@some-company-cloud.com", + "modifyTime": "2020-12-16T11:52:24.477Z", + "modifyUser": "some-user@some-company-cloud.com", + "resourceCurationStatus": "namespace:reference-data--ResourceCurationStatus:CREATED:", + "resourceLifecycleStatus": "namespace:reference-data--ResourceLifecycleStatus:LOADING:", + "resourceSecurityClassification": "namespace:reference-data--ResourceSecurityClassification:SomeUniqueResourceSecurityClassificationID:", + "ancestry": { + "parents": [] + }, + "meta": [ + { + "kind": "CRS", + "name": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "persistableReference": "{\"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]]\"}", + "coordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:", + "propertyNames": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + ], + "source": "Example Data Source", + "existenceKind": "namespace:reference-data--ExistenceKind:Prototype:", + "data": { + "Name": "Dataset X221/15", + "Description": "As originally delivered by ACME.com.", + "Source": "ACME.com", + "TotalSize": "13245217273", + "EncodingFormatTypeID": "namespace:reference-data--EncodingFormatType:UTF-8:", + "MimeType": "namespace:reference-data--MimeType:application/geo+json:", + "Endian": "BIG", + "DatasetProperties": {} + } + }, + { + "id": "surrogate-key:file-2", + "kind": "osdu:wks:dataset--GenericDataset:1.0.0", + "version": 1562066009929332, + "acl": { + "owners": [ + "someone@company.com" + ], + "viewers": [ + "someone@company.com" + ] + }, + "legal": { + "legaltags": [ + "Example legaltags" + ], + "otherRelevantDataCountries": [ + "US" + ], + "status": "compliant" + }, + "resourceHomeRegionID": "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:", + "resourceHostRegionIDs": [ + "namespace:reference-data--OSDURegion:SomeUniqueOSDURegionID:" + ], + "createTime": "2020-12-16T11:46:20.163Z", + "createUser": "some-user@some-company-cloud.com", + "modifyTime": "2020-12-16T11:52:24.477Z", + "modifyUser": "some-user@some-company-cloud.com", + "resourceCurationStatus": "namespace:reference-data--ResourceCurationStatus:CREATED:", + "resourceLifecycleStatus": "namespace:reference-data--ResourceLifecycleStatus:LOADING:", + "resourceSecurityClassification": "namespace:reference-data--ResourceSecurityClassification:SomeUniqueResourceSecurityClassificationID:", + "ancestry": { + "parents": [] + }, + "meta": [ + { + "kind": "CRS", + "name": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]", + "persistableReference": "{\"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]]\"}", + "coordinateReferenceSystemID": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:", + "propertyNames": [ + "KickOffPosition.X", + "KickOffPosition.Y" + ] + } + ], + "source": "Example Data Source", + "existenceKind": "namespace:reference-data--ExistenceKind:Prototype:", + "data": { + "Name": "Dataset X221/15", + "Description": "As originally delivered by ACME.com.", + "Source": "ACME.com", + "TotalSize": "13245217273", + "EncodingFormatTypeID": "namespace:reference-data--EncodingFormatType:UTF-8:", + "MimeType": "namespace:reference-data--MimeType:application/geo+json:", + "Endian": "BIG", + "DatasetProperties": {} + } + } + ] + } +} diff --git a/tests/plugin-unit-tests/data/manifests/schema_Manifest.1.0.0.json b/tests/plugin-unit-tests/data/manifests/schema_Manifest.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..facd766ba5bac7cddeb2b7534fa0468f5d2a2477 --- /dev/null +++ b/tests/plugin-unit-tests/data/manifests/schema_Manifest.1.0.0.json @@ -0,0 +1,55 @@ +{ + "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", + "$id": "https://schema.osdu.opengroup.org/json/manifest/Manifest.1.0.0.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "Load Manifest Schema", + "description": "Load manifest applicable for all types defined as 'kind', i.e. registered as schemas with the Schema Service. It supports loading of individual 'records' of any group-type or combinations. The load sequence follows a well-defined sequence. The 'ReferenceData' array is processed first (if populated). The 'MasterData' array is processed second (if populated) second. The 'Data' structure is processed last (if populated). Inside the 'Data' property the 'Files' array is processed first, followed by the 'WorkProductComponents' array, the 'WorkProduct' is processed last. Any arrays are ordered. should there be interdependencies, the dependent items must be placed behind their relationship targets, e.g. a master-data Well record must placed in the 'MasterData' array before its Wellbores.", + "type": "object", + "properties": { + "kind": { + "description": "The schema identification for the manifest record following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.", + "title": "Manifest Kind", + "type": "string", + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.\\/]+:[0-9]+.[0-9]+.[0-9]+$", + "example": "osdu:wks:Manifest:1.0.0" + }, + "ReferenceData": { + "description": "Reference-data are submitted as an array of records.", + "type": "array", + "items": { + "$ref": "GenericReferenceData.1.0.0.json" + } + }, + "MasterData": { + "description": "Master-data are submitted as an array of records.", + "type": "array", + "items": { + "$ref": "GenericMasterData.1.0.0.json" + } + }, + "Data": { + "description": "Manifest schema for work-product, work-product-component, dataset ensembles. The items in 'Datasets' are processed first since they are referenced by 'WorkProductComponents' ('data.DatasetIDs[]' and 'data.Artefacts[].ResourceID'). The WorkProduct is processed last collecting the WorkProductComponents.", + "type": "object", + "properties": { + "WorkProduct": { + "description": "The work-product component capturing the work-product-component records belonging to this loading/ingestion transaction.", + "$ref": "GenericWorkProduct.1.0.0.json" + }, + "WorkProductComponents": { + "description": "The list of work-product-components records. The record ids are internal surrogate keys enabling the association of work-product-component records with the work-product records.", + "type": "array", + "items": { + "$ref": "GenericWorkProductComponent.1.0.0.json" + } + }, + "Datasets": { + "description": "The list of 'Datasets' or data containers holding the actual data. The record ids are usually internal surrogate keys enabling the association of dataset records with work-product-component records, namely via 'DatasetIDs' and 'Artefacts.ResourceID' (both referring to 'dataset' group-type entity types).", + "type": "array", + "items": { + "$ref": "GenericDataset.1.0.0.json" + } + } + } + } + } +} diff --git a/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json index b38869dbedaea6272fba4c8d87f9fc1d3d09889e..9f1bbd688339debda72cbbd21155a3776c441c35 100644 --- a/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json +++ b/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json @@ -7,10 +7,13 @@ }, "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - { + "manifest": { + "kind": "test:test:Manifest:1.0.0", + "ReferenceData": [], + "MasterData": [ + { "id": "opendes:master-data/Wellbore:350112350400", - "kind": "opendes:osdu:Wellbore:0.3.0", + "kind": "opendes:osdu:TestMaster:0.3.0", "groupType": "master-data", "version": 1, "acl": { @@ -40,98 +43,12 @@ "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:", "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:", "data": { - "FacilityTypeID": "srn:opendes:reference-data/FacilityType:Wellbore:", - "FacilityOperator": [ - { - "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:" - } - ], - "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:", - "SpatialLocation": [ - { - "Coordinates": [ - { - "x": -98.580887, - "y": 35.6381829999999 - } - ], - "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Point:", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:", - "HorizontalCRSID": "srn:opendes:reference-data/HorizontalCRS:NAD27:", - "HeightAboveGroundLevelUOMID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:" - } - ], - "OperatingEnvironmentID": "srn:opendes:reference-data/OperatingEnvironment:onshore:", - "FacilityName": "IRETA 1-4-9XH", - "FacilityNameAlias": [ - { - "AliasName": " IRETA 1-4-9XH", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:Name:" - }, - { - "AliasName": "350112350400", - "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:UWBI:" - } - ], - "FacilityEvent": [ - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD:", - "EffectiveDateTime": "2015-03-11T00:00:00-05:00" - }, - { - "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:", - "EffectiveDateTime": "2015-05-18T00:00:00-06:00" - } - ], - "WellID": "srn:opendes:master-data/Well:3501123504:", - "SequenceNumber": 1, - "VerticalMeasurements": [ - { - "VerticalMeasurementID": "TD_1", - "VerticalMeasurement": 0, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Measured Depth:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalReferenceID": "Drill Floor" - }, - { - "VerticalMeasurementID": "TD_2", - "VerticalMeasurement": 0, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:True Vertical Depth:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalReferenceID": "Drill Floor" - }, - { - "VerticalMeasurementID": "Elev_1", - "VerticalMeasurement": 1636, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Drill Floor:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:" - }, - { - "VerticalMeasurementID": "Elev_2", - "VerticalMeasurement": 1606, - "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Ground Level:", - "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:", - "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:", - "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:" - } - ], - "TrajectoryTypeID": "srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:", - "DefaultVerticalMeasurementID": "", - "GeographicBottomHoleLocation": { - "Coordinates": [ - { - "x": -98.580887, - "y": 35.6381829999999 - } - ] - } - } + "SequenceNumber": 1 + }, + "schema": "test:test:GenericMasterData:1.0.0" } - - ], + ], + "Data": {} + }, "WorkflowID": "foo" } diff --git a/tests/plugin-unit-tests/data/master/schema_GenericMasterData.1.0.0.json b/tests/plugin-unit-tests/data/master/schema_GenericMasterData.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..57e53f592c2219be17d1f17174ff1692e0406d51 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/schema_GenericMasterData.1.0.0.json @@ -0,0 +1,96 @@ +{ + "$id": "https://schema.osdu.opengroup.org/json/master-data/GenericMasterData.1.0.0.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "x-osdu-schema-source": "osdu:wks:master-data--GenericMasterData:1.0.0", + "title": "GenericMasterData", + "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.", + "type": "object", + "properties": { + "id": { + "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" + }, + "kind": { + "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", + "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.\\/]+:[0-9]+.[0-9]+.[0-9]+$", + "example": "osdu:wks:master-data--GenericMasterData:1.0.0" + }, + "version": { + "description": "The version number of this OSDU resource; set by the framework.", + "title": "Version Number", + "type": "integer", + "format": "int64", + "example": 1562066009929332 + }, + "acl": { + "description": "The access control tags associated with this entity.", + "title": "Access Control List" + }, + "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" + }, + "tags": { + "title": "Tag Dictionary", + "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.", + "type": "object", + "additionalProperties": { + "type": "string" + }, + "example": { + "NameOfKey": "String value" + } + }, + "createTime": { + "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", + "format": "date-time", + "example": "2020-12-16T11:46:20.163Z" + }, + "createUser": { + "title": "Resource Object Creation User Reference", + "description": "The user reference, which created the first version of this resource object. Set by the System.", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "modifyTime": { + "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", + "format": "date-time", + "example": "2020-12-16T11:52:24.477Z" + }, + "modifyUser": { + "title": "Resource Object Version Creation User Reference", + "description": "The user reference, which created this version of this resource object. Set by the System.", + "type": "string", + "example": "some-user@some-company-cloud.com" + }, + "ancestry": { + "description": "The links to data, which constitute the inputs.", + "title": "Ancestry" + }, + "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": { + } + }, + "data": { + "type": "object", + "description": "The data block containing the kind-specific payload. The actual schema is defined by the 'kind', resolvable via the Schema Service." + } + }, + "required": [ + "id", + "kind", + "acl", + "legal" + ] +} diff --git a/tests/plugin-unit-tests/data/master/schema_TestMaster.json b/tests/plugin-unit-tests/data/master/schema_TestMaster.json new file mode 100644 index 0000000000000000000000000000000000000000..8e41586e9fafdbfc88dd10c7da3f72aa9e1f37f7 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/schema_TestMaster.json @@ -0,0 +1,13 @@ +{ + "$id": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "Master", + "description": "A hole in the ground extending from a point at the earth's surface to the maximum point of penetration.", + "type": "object", + "properties": { + "SequenceNumber": { + "description": "A number that indicates the order in which wellbores were drilled.", + "type": "integer" + } + } +} diff --git a/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json b/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json index 43f7e6e07f225840963f209ca2f99270d5edd450..3df9b5c186c5eeeb09183a3c5a323359ccb74952 100644 --- a/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json +++ b/tests/plugin-unit-tests/data/master/schema_Wellbore.3.0.0.json @@ -2,7 +2,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "A hole in the ground extending from a point at the earth's surface to the maximum point of penetration.", - "additionalProperties": false, + "additionalProperties": true, "title": "Wellbore", "type": "object", "definitions": { @@ -82,7 +82,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "Legal meta data like legal tags, relevant other countries, legal status.", - "additionalProperties": false, + "additionalProperties": true, "title": "Legal Meta Data", "type": "object", "properties": { @@ -118,7 +118,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "The access control tags associated with this entity.", - "additionalProperties": false, + "additionalProperties": true, "title": "Access Control List", "type": "object", "properties": { diff --git a/tests/plugin-unit-tests/data/master/traversal_Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/traversal_Wellbore.0.3.0.json new file mode 100644 index 0000000000000000000000000000000000000000..9a4dd216dd290d36b62b5892508ed69987d993f8 --- /dev/null +++ b/tests/plugin-unit-tests/data/master/traversal_Wellbore.0.3.0.json @@ -0,0 +1,41 @@ +[ + { + "entity": { + "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" + }, + "schema": "GenericMasterData.1.0.0.json" + } +] diff --git a/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json b/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json index 9c4035d36cadd21c16e54b8a25b0ad1844a6f344..4a7da0664e5afed083e5c03b625d5473048af078 100644 --- a/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json +++ b/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json @@ -7,8 +7,11 @@ }, "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json", "$filename": "load_Wellbore.1.0.0_350112350400.json", - "manifest": [ - { + "manifest": + { + "ReferenceData": [], + "MasterData": [], + "Data": { "WorkProduct": { "kind": "opendes:osdu:WorkProduct:1.0.0", "groupType": "work-product", @@ -351,7 +354,7 @@ ] } ], - "Files": [ + "Datasets": [ { "kind": "opendes:osdu:File:1.0.0", "groupType": "file", @@ -374,9 +377,14 @@ }, "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", "data": { + "Name": "RequiredName", "SchemaFormatTypeID": "srn:opendes:reference-data/SchemaFormatType:SEG-Y Seismic Trace Data:", - "PreLoadFilePath": "test", - "FileSource": "", + "DatasetProperties": { + "FileSourceInfo": { + "PreloadFilePath": "test", + "FileSource": "" + } + }, "FileSize": 277427976, "EncodingFormatTypeID": "srn:opendes:reference-data/EncodingFormatType:segy:", "Endian": "BIG", @@ -418,6 +426,6 @@ } ] } - ], + }, "WorkflowID": "foo" } diff --git a/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json b/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json index 0c39e27d8b313625fb546065c412b43440e81ee2..1eb8d59a3464b2ca132c50457abe504a27a78b16 100644 --- a/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json +++ b/tests/plugin-unit-tests/data/workProduct/record_SeismicTraceData.json @@ -20,50 +20,50 @@ "kind": "opendes:osdu:File:1.0.0", "id": "test_id", "data": { - "groupType": "file", - "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", - "data": { - "SchemaFormatTypeID": "srn:opendes:reference-data/SchemaFormatType:SEG-Y Seismic Trace Data:", - "PreLoadFilePath": "test", - "FileSource": "test", - "FileSize": 277427976, - "EncodingFormatTypeID": "srn:opendes:reference-data/EncodingFormatType:segy:", - "Endian": "BIG", - "Checksum": "c9df7234d5d0a7a2c2676ee2e2aa48b6", - "VectorHeaderMapping": [ - { - "KeyName": "srn:opendes:reference-data/HeaderKeyName:INLINE:", - "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", - "WordWidth": 4, - "Position": 189 - }, - { - "KeyName": "srn:opendes:reference-data/HeaderKeyName:CROSSLINE:", - "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", - "WordWidth": 4, - "Position": 193 - }, - { - "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPX:", - "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", - "WordWidth": 4, - "Position": 181, - "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", - "ScalarIndicator": "OVERRIDE", - "ScalarOverride": 100.0 - }, - { - "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPY:", - "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", - "WordWidth": 4, - "Position": 185, - "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", - "ScalarIndicator": "OVERRIDE", - "ScalarOverride": 100.0 - } - ] + "Name": "RequiredName", + "SchemaFormatTypeID": "srn:opendes:reference-data/SchemaFormatType:SEG-Y Seismic Trace Data:", + "DatasetProperties": { + "FileSourceInfo": { + "FileSource": "/test/source_file", + "PreloadFilePath": "test" + } }, - "AssociativeID": "f-1" + "FileSize": 277427976, + "EncodingFormatTypeID": "srn:opendes:reference-data/EncodingFormatType:segy:", + "Endian": "BIG", + "Checksum": "c9df7234d5d0a7a2c2676ee2e2aa48b6", + "VectorHeaderMapping": [ + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:INLINE:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 189 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CROSSLINE:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 193 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPX:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 181, + "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", + "ScalarIndicator": "OVERRIDE", + "ScalarOverride": 100.0 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPY:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 185, + "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", + "ScalarIndicator": "OVERRIDE", + "ScalarOverride": 100.0 + } + ] } }, { diff --git a/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json index ba5e926ebb17d760cbf7b8b76270b42150103a02..ee12e41f1a9b4ba03269641c5ddde243797566ea 100644 --- a/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json +++ b/tests/plugin-unit-tests/data/workProduct/schema_File.1.0.0.json @@ -2,7 +2,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "The generic file entity.", - "additionalProperties": false, + "additionalProperties": true, "title": "File", "type": "object", "definitions": { @@ -82,7 +82,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "Legal meta data like legal tags, relevant other countries, legal status.", - "additionalProperties": false, + "additionalProperties": true, "title": "Legal Meta Data", "type": "object", "properties": { @@ -118,7 +118,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "The access control tags associated with this entity.", - "additionalProperties": false, + "additionalProperties": true, "title": "Access Control List", "type": "object", "properties": { diff --git a/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json index 7a7e2bd79ef6d5b210cf75b4779726057a87ca43..718e04888be34581fd1130eca2b4a401044cabbf 100644 --- a/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json +++ b/tests/plugin-unit-tests/data/workProduct/schema_SeismicTraceData.1.0.0.json @@ -2,7 +2,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "A single logical dataset containing seismic samples.", - "additionalProperties": false, + "additionalProperties": true, "title": "SeismicTraceData", "type": "object", "definitions": { @@ -82,7 +82,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "Legal meta data like legal tags, relevant other countries, legal status.", - "additionalProperties": false, + "additionalProperties": true, "title": "Legal Meta Data", "type": "object", "properties": { @@ -118,7 +118,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "The access control tags associated with this entity.", - "additionalProperties": false, + "additionalProperties": true, "title": "Access Control List", "type": "object", "properties": { diff --git a/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json index f2c69e41474742506861416dc4ea074fff171b6e..ab49b4edf52ae75773851b895b87b197ffddba5c 100644 --- a/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json +++ b/tests/plugin-unit-tests/data/workProduct/schema_WorkProduct.1.0.0.json @@ -2,7 +2,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "A collection of work product components such as might be produced by a business activity and which is delivered to the data platform for loading.", - "additionalProperties": false, + "additionalProperties": true, "title": "WorkProduct", "type": "object", "definitions": { @@ -82,7 +82,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "Legal meta data like legal tags, relevant other countries, legal status.", - "additionalProperties": false, + "additionalProperties": true, "title": "Legal Meta Data", "type": "object", "properties": { @@ -118,7 +118,7 @@ "x-osdu-license": "Copyright 2020, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$schema": "http://json-schema.org/draft-07/schema#", "description": "The access control tags associated with this entity.", - "additionalProperties": false, + "additionalProperties": true, "title": "Access Control List", "type": "object", "properties": { diff --git a/tests/plugin-unit-tests/data/workProduct/traversal_SeismicTraceData.1.0.0.json b/tests/plugin-unit-tests/data/workProduct/traversal_SeismicTraceData.1.0.0.json new file mode 100644 index 0000000000000000000000000000000000000000..a8ba5d61e1314c7000380be0a2baf0c1b3beacd8 --- /dev/null +++ b/tests/plugin-unit-tests/data/workProduct/traversal_SeismicTraceData.1.0.0.json @@ -0,0 +1,420 @@ +[ + { + "entity": { + "kind": "opendes:osdu:WorkProduct:1.0.0", + "groupType": "work-product", + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "legal": { + "legaltags": [ + "legaltag" + ], + "otherRelevantDataCountries": [ + "NO", + "US" + ] + }, + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", + "data": { + "Name": "ST0202R08_PS_PSDM_RAW_PP_TIME.MIG_RAW", + "Description": "Seismic Trace Data" + }, + "ComponentsAssociativeIDs": [ + "wpc-1" + ] + }, + "schema": "GenericWorkProduct.1.0.0.json" + }, + { + "entity": { + "kind": "opendes:osdu:SeismicTraceData:1.0.0", + "groupType": "work-product-component", + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "legal": { + "legaltags": [ + "legaltag" + ], + "otherRelevantDataCountries": [ + "NO", + "US" + ] + }, + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", + "meta": [ + { + "kind": "Unit", + "name": "ms", + "persistableReference": "{\"abcd\":{\"a\":0.0,\"b\":0.001,\"c\":1.0,\"d\":0.0},\"symbol\":\"ms\",\"baseMeasurement\":{\"ancestry\":\"T\",\"type\":\"UM\"},\"type\":\"UAD\"}", + "unitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:Energistics_UoM_ms:", + "propertyNames": [ + "StartTime", + "EndTime" + ] + }, + { + "kind": "Unit", + "name": "Amplitude", + "persistableReference": "{\"abcd\":{\"a\":0.0,\"b\":1.0,\"c\":1.0,\"d\":0.0},\"symbol\":\"Euc\",\"baseMeasurement\":{\"ancestry\":\"1\",\"type\":\"UM\"},\"type\":\"UAD\"}", + "unitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:Energistics_UoM_Euc:", + "propertyNames": [ + "RangeAmplitudeMax", + "RangeAmplitudeMin" + ] + } + ], + "data": { + "Name": "ST0202R08_PS_PSDM_RAW_PP_TIME.MIG_RAW", + "Description": "Seismic Trace Data", + "SpatialArea": { + "AsIngestedCoordinates": { + "type": "AnyCrsFeatureCollection", + "CoordinateReferenceSystemID": "srn:opendes:reference-data/CoordinateReferenceSystem:BoundCRS.SLB.23031.1613:", + "persistableReferenceCRS": "{\"lateBoundCRS\":{\"wkt\":\"PROJCS[\\\"ED_1950_UTM_Zone_31N\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Transverse_Mercator\\\"],PARAMETER[\\\"False_Easting\\\",500000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",3.0],PARAMETER[\\\"Scale_Factor\\\",0.9996],PARAMETER[\\\"Latitude_Of_Origin\\\",0.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",23031]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_UTM_Zone_31N\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"23031\"},\"type\":\"LBC\"},\"singleCT\":{\"wkt\":\"GEOGTRAN[\\\"ED_1950_To_WGS_1984_24\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],METHOD[\\\"Position_Vector\\\"],PARAMETER[\\\"X_Axis_Translation\\\",-90.365],PARAMETER[\\\"Y_Axis_Translation\\\",-101.13],PARAMETER[\\\"Z_Axis_Translation\\\",-123.384],PARAMETER[\\\"X_Axis_Rotation\\\",0.333],PARAMETER[\\\"Y_Axis_Rotation\\\",0.077],PARAMETER[\\\"Z_Axis_Rotation\\\",0.894],PARAMETER[\\\"Scale_Difference\\\",1.994],AUTHORITY[\\\"EPSG\\\",1613]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_To_WGS_1984_24\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"1613\"},\"type\":\"ST\"},\"ver\":\"PE_10_3_1\",\"name\":\"ED50 * EPSG-Nor S62 2001 / UTM zone 31N [23031,1613]\",\"authCode\":{\"auth\":\"SLB\",\"code\":\"23031024\"},\"type\":\"EBC\"}", + "features": [ + { + "type": "AnyCrsFeature", + "properties": null, + "geometry": { + "type": "AnyCrsPolygon", + "coordinates": [ + [ + [ + 438727.125, + 6475514.5 + ], + [ + 439888.34375, + 6480172.0 + ], + [ + 432562.59375, + 6481998.5 + ], + [ + 431401.375, + 6477341.0 + ], + [ + 438727.125, + 6475514.5 + ] + ] + ] + } + } + ] + }, + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "properties": null, + "geometry": { + "type": "Polygon", + "coordinates": [ + [ + [ + 1.9496878, + 58.4141503 + ], + [ + 1.9683366, + 58.4561357 + ], + [ + 1.8422866, + 58.4714655 + ], + [ + 1.8237804, + 58.4294624 + ], + [ + 1.9496878, + 58.4141503 + ] + ] + ] + } + } + ] + }, + "OperationsApplied": [ + "AsIngestedCoordinates converted to Wgs84Coordinates: Input CRS EPSG 23031 (ED50 / UTM zone 31N) to Target CRS EPSG 4326 (WGS84) using CT EPSG 1613 (ED50 to WGS 84 (24) - Norway - offshore south of 62°N - North Sea.)" + ], + "SpatialParameterTypeID": "srn:opendes:reference-data/SpatialParameterType:Outline:", + "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Polygon:" + }, + "LiveTraceOutline": { + "AsIngestedCoordinates": { + "type": "AnyCrsFeatureCollection", + "CoordinateReferenceSystemID": "srn:opendes:reference-data/CoordinateReferenceSystem:BoundCRS.SLB.23031.1613:", + "persistableReferenceCRS": "{\"lateBoundCRS\":{\"wkt\":\"PROJCS[\\\"ED_1950_UTM_Zone_31N\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Transverse_Mercator\\\"],PARAMETER[\\\"False_Easting\\\",500000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",3.0],PARAMETER[\\\"Scale_Factor\\\",0.9996],PARAMETER[\\\"Latitude_Of_Origin\\\",0.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",23031]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_UTM_Zone_31N\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"23031\"},\"type\":\"LBC\"},\"singleCT\":{\"wkt\":\"GEOGTRAN[\\\"ED_1950_To_WGS_1984_24\\\",GEOGCS[\\\"GCS_European_1950\\\",DATUM[\\\"D_European_1950\\\",SPHEROID[\\\"International_1924\\\",6378388.0,297.0]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],METHOD[\\\"Position_Vector\\\"],PARAMETER[\\\"X_Axis_Translation\\\",-90.365],PARAMETER[\\\"Y_Axis_Translation\\\",-101.13],PARAMETER[\\\"Z_Axis_Translation\\\",-123.384],PARAMETER[\\\"X_Axis_Rotation\\\",0.333],PARAMETER[\\\"Y_Axis_Rotation\\\",0.077],PARAMETER[\\\"Z_Axis_Rotation\\\",0.894],PARAMETER[\\\"Scale_Difference\\\",1.994],AUTHORITY[\\\"EPSG\\\",1613]]\",\"ver\":\"PE_10_3_1\",\"name\":\"ED_1950_To_WGS_1984_24\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"1613\"},\"type\":\"ST\"},\"ver\":\"PE_10_3_1\",\"name\":\"ED50 * EPSG-Nor S62 2001 / UTM zone 31N [23031,1613]\",\"authCode\":{\"auth\":\"SLB\",\"code\":\"23031024\"},\"type\":\"EBC\"}", + "features": [ + { + "type": "AnyCrsFeature", + "properties": null, + "geometry": { + "type": "AnyCrsPolygon", + "coordinates": [ + [ + [ + 438727.125, + 6475514.5 + ], + [ + 439888.34375, + 6480172.0 + ], + [ + 432562.59375, + 6481998.5 + ], + [ + 431401.375, + 6477341.0 + ], + [ + 438727.125, + 6475514.5 + ] + ] + ] + } + } + ] + }, + "Wgs84Coordinates": { + "type": "FeatureCollection", + "features": [ + { + "type": "Feature", + "properties": null, + "geometry": { + "type": "Polygon", + "coordinates": [ + [ + [ + 1.9496878, + 58.4141503 + ], + [ + 1.9683366, + 58.4561357 + ], + [ + 1.8422866, + 58.4714655 + ], + [ + 1.8237804, + 58.4294624 + ], + [ + 1.9496878, + 58.4141503 + ] + ] + ] + } + } + ] + }, + "OperationsApplied": [ + "AsIngestedCoordinates converted to Wgs84Coordinates: Input CRS EPSG 23031 (ED50 / UTM zone 31N) to Target CRS EPSG 4326 (WGS84) using CT EPSG 1613 (ED50 to WGS 84 (24) - Norway - offshore south of 62°N - North Sea.)" + ], + "SpatialParameterTypeID": "srn:opendes:reference-data/SpatialParameterType:Outline:", + "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Polygon:" + }, + "PrincipalAcquisitionProjectID": "srn:opendes:master-data/SeismicAcquisitionProject:ST0202R08:", + "ProcessingProjectID": "srn:opendes:master-data/SeismicProcessingProject:ST0202R08:", + "SeismicTraceDataDimensionalityTypeID": "srn:opendes:reference-data/SeismicTraceDataDimensionalityType:3D:", + "SeismicDomainTypeID": "srn:opendes:reference-data/SeismicDomainType:Time:", + "SeismicMigrationTypeID": "srn:opendes:reference-data/SeismicMigrationType:Prestack Depth - Kirchhoff:", + "SeismicStackingTypeID": "srn:opendes:reference-data/SeismicStackingType:Full:", + "SeismicFilteringTypeID": "srn:opendes:reference-data/SeismicFilteringType:Tau-P:", + "Phase": "0", + "Polarity": "Normal", + "SampleInterval": 4.0, + "SampleCount": 1126, + "Difference": false, + "StartTime": 0.0, + "EndTime": 4500.0, + "TraceCount": 58479, + "TraceLength": 4500.0, + "TraceDomainUOM": "srn:opendes:reference-data/UnitOfMeasure:ms:", + "InlineMin": 9985, + "InlineMax": 10369, + "CrosslineMin": 1932, + "CrosslineMax": 2536, + "InlineIncrement": 2, + "CrosslineIncrement": 2, + "Precision": { + "WordFormat": "srn:opendes:reference-data/WordFormatType:IBM_FLOAT:", + "WordWidth": 4 + }, + "ProcessingParameters": [ + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:REFORMAT:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:NAVIGATION MERGE:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:TRACE EDITING:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:DESIGNATURE/ZEROPHASE:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:TIDAL STATICS:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:RESAMPLE:", + "ProcessingParameterValue": "4 MS" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:GEOPHONE MATCHING CORRECTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:GEOPHONE ORIENTATION CORRECTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:ROTATION TO RADIAL:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:NOISE BAND EDITING:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:RECEIVER SHEAR STATIC CORRECTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:TAUP DECONVOLUTION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:3D KIRCHHOFF DEPTH MIGRATION:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:VEL ANALYSIS AND RMO CORRECTION:", + "ProcessingParameterValue": "200M X 200M" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:CONVERSION TO PP TWT:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:PRE-RADON MILD DIP FILTER:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:PARABOLIC RADON TRANSFORM DEMULTIPLE:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:ANGLE MUTE:", + "ProcessingParameterValue": "3-37 DEGREES" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:STACK:" + }, + { + "ProcessingParameterTypeID": "srn:opendes:reference-data/ProcessingParameterType:OUTPUT SEGY:" + } + ], + "TextualFileHeader": [ + "C 1 CLIENT=STATOIL, PROCESSED BY=WESTERNGECO C 2 LINE ST0202D04-9985 C 3 SURVEY:ST0202 , AREA: VOLVE 15/9 C 4 DATAFORMAT: SEG-Y, DATE:02012008 C 5 DATATYPE: RAW PS PSDM STACK IN PP TIME C 6 DATA SHOT BY/VESSEL:GECO ANGLER, CABLE LENGTH:6000 M NO OF GROUPS: 240x4 C 7 NO OF CABLES 2, SOURCE:2, POP INTERVAL 25.0 M (FLIP-FLOP) C 8 RCV LINE SPACING: 400 M, SRC LINE SPACING: 100 M, RECORD LENGTH: 10.2 S C 9 GEODETIC DATUM: ED-50, SPHEROID: INTERNAT., PROJECTION: UTM C10 CENTR. MERID.: 03, UTM ZONE: 31 N, FALSE EASTING: 500000 C11 FIRST SAMPLE= 0, LAST SAMPLE= 4500, SAMPLE INTERVAL= 4 MS C12 DIST. BETWEEN INLINES=25.0M, XLINES=25.0M BIN SIZE (I x X):12.5M x 12.5M C13 INLINE DIRECTION (GRID): 284.0000000 DEGREES (CLOCKWISE FROM NORTH); C14 X-LINE DIRECTION (GRID): 014.0000000 DEGREES (CLOCKWISE FROM NORTH); C15 ORIGO(1.1) UTMX:431955.70E; UTMY:6348582.15N ; C16 DATA RANGE INLINES=9985-10369 (EVERY 2ND), X-LINES=1932-2536 (EVERY 2ND) C17 CORNER1:3D INLINE 9985, 3D XLINE 1932, UTM-X 438727.0, UTM-Y 6475514.4 C18 CORNER2:3D INLINE 9985, 3D XLINE 2536, UTM-X 431401.3, UTM-Y 6477341.0 C19 CORNER3:3D INLINE 10369, 3D XLINE 2536, UTM-X 432562.5, UTM-Y 6481998.4 C20 CORNER4:3D INLINE 10369, 3D XLINE 1932, UTM-X 439888.3, UTM-Y 6480171.9 C21 LIVE DATA POLYGON: (9985,1932);(9985,2536);(10369,2536);(10369,1932); C22 NAVIGATION SOURCE: P1/90 UKOOA BIN CENTER CELL GRID, C23 PROCESSING SEQUENCE: C24 1) REFORMAT 2) NAVIGATION MERGE 3) TRACE EDITING 4) DESIGNATURE/ZEROPHASE C25 5) TIDAL STATICS 6) RESAMPLE 4MS 7) GEOPHONE MATCHING CORRECTION C26 8) GEOPHONE ORIENTATION CORRECTION 9) ROTATION TO RADIAL C27 10) NOISE BAND EDITING 11) RECEIVER SHEAR STATIC CORRECTION C28 12) TAUP DECONVOLUTION 13) 3D KIRCHHOFF DEPTH MIGRATION C29 14) VEL ANALYSIS AND RMO CORRECTION 200M X 200M 15) CONVERSION TO PP TWT C30 16) PRE-RADON MILD DIP FILTER 17) PARABOLIC RADON TRANSFORM DEMULTIPLE C31 18) 3-37 DEGREE ANGLE MUTE 19) STACK 20) OUTPUT TO SEGY FORMAT C32 C33 C34 C35 HEADER WORD POSITIONS: C36 INLINE: 189-192 ; X-LINE: 193-196; C37 BINX (CDPX): 181-184, BINY (CDPY): 185-188, C38 MERID.: 3.0E, SPHEROID: INT.; ROTATION (AMS): 1245600000, C39 A POSITIVE SAMPLE CORRESPONDS TO A INCREASE IN ACOUSTIC IMPEDANCE. C40 END EBCDIC. " + ], + "RangeAmplitudeMax": 0.07441109418869019, + "RangeAmplitudeMin": -0.10446560382843018 + }, + "AssociativeID": "wpc-1", + "FileAssociativeIDs": [ + "f-1" + ] + }, + "schema": "GenericWorkProductComponent.1.0.0.json" + }, + { + "entity": { + "kind": "opendes:osdu:File:1.0.0", + "groupType": "file", + "acl": { + "owners": [ + "ownergroup@testcompany.com" + ], + "viewers": [ + "viewgroup@testcompany.com" + ] + }, + "legal": { + "legaltags": [ + "legaltag" + ], + "otherRelevantDataCountries": [ + "NO", + "US" + ] + }, + "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:RESTRICTED:", + "data": { + "Name": "RequiredName", + "SchemaFormatTypeID": "srn:opendes:reference-data/SchemaFormatType:SEG-Y Seismic Trace Data:", + "DatasetProperties": { + "FileSourceInfo": { + "FileSource": "", + "PreloadFilePath": "test" + } + }, + "FileSize": 277427976, + "EncodingFormatTypeID": "srn:opendes:reference-data/EncodingFormatType:segy:", + "Endian": "BIG", + "Checksum": "c9df7234d5d0a7a2c2676ee2e2aa48b6", + "VectorHeaderMapping": [ + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:INLINE:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 189 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CROSSLINE:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 193 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPX:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 181, + "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", + "ScalarIndicator": "OVERRIDE", + "ScalarOverride": 100.0 + }, + { + "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPY:", + "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:", + "WordWidth": 4, + "Position": 185, + "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:", + "ScalarIndicator": "OVERRIDE", + "ScalarOverride": 100.0 + } + ] + }, + "AssociativeID": "f-1" + }, + "schema": "GenericDataset.1.0.0.json" + } +] diff --git a/tests/plugin-unit-tests/file_paths.py b/tests/plugin-unit-tests/file_paths.py index 295877c85d0fe4adfce1621cb622a2fce1149d25..e2368312f0c3a9f31beea5285c927444dd4ccdab 100644 --- a/tests/plugin-unit-tests/file_paths.py +++ b/tests/plugin-unit-tests/file_paths.py @@ -17,17 +17,25 @@ 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_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" 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" +SCHEMA_TEST_MASTERDATA_PATH = f"{DATA_PATH_PREFIX}/master/schema_TestMaster.json" +TRAVERSAL_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/traversal_Wellbore.0.3.0.json" MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/SeismicTraceData.json" +TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/traversal_SeismicTraceData.1.0.0.json" SCHEMA_FILE_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_File.1.0.0.json" SCHEMA_WORK_PRODUCT_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_WorkProduct.1.0.0.json" SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_SeismicTraceData.1.0.0.json" RECORD_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/record_SeismicTraceData.json" MANIFEST_EMPTY_PATH = f"{DATA_PATH_PREFIX}/invalid/EmptyManifest.json" +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" diff --git a/tests/plugin-unit-tests/test_file_handler.py b/tests/plugin-unit-tests/test_file_handler.py new file mode 100644 index 0000000000000000000000000000000000000000..ebad2d7bb696539c22da4bdf2bc6a3775419b6df --- /dev/null +++ b/tests/plugin-unit-tests/test_file_handler.py @@ -0,0 +1,163 @@ +# Copyright 2020 Google LLC +# Copyright 2020 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import http +import io +import json +import os +import sys + +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") + +from libs.exceptions import FileSourceError, GCSObjectURIError, InvalidFileRecordData +import pytest +import requests +import responses +import tenacity +from file_paths import RECORD_SEISMIC_TRACE_DATA_VALID_PATH +from libs.context import Context +from libs.refresh_token import AirflowTokenRefresher +from libs.handle_file import FileDownloadUrlResponse, FileUploadUrlResponse, GCSFileHandler + + +class TestFileHandler: + + BASE_TEST_URL = "http://file_service_url" + PARTITION_ID = "test_partition_id" + + @pytest.fixture() + def file_handler(self, monkeypatch): + context = Context(data_partition_id=self.PARTITION_ID, app_key="") + file_handler = GCSFileHandler(self.BASE_TEST_URL, AirflowTokenRefresher(), context) + monkeypatch.setattr( + file_handler, + "_get_upload_signed_url", + lambda *args, **kwargs: FileUploadUrlResponse( + "test_id", "test_signed_url", "test_file_source")) + monkeypatch.setattr( + file_handler, + "_upload_file_request", + lambda *args, **kwargs: None) + return file_handler + + def test_get_file_from_bucket( + self, + monkeypatch, + file_handler: GCSFileHandler + ): + file = io.RawIOBase() + monkeypatch.setattr(file_handler, "get_file_from_bucket", + lambda *args, **kwargs: (file, "test")) + file_handler.upload_file("gs://test/test") + + @pytest.mark.parametrize( + "file_path", + [ + pytest.param("gs://test"), + pytest.param("://test"), + pytest.param("test"), + ] + ) + def test_invalid_gcs_object_uri(self, file_handler: GCSFileHandler, + file_path: str): + with pytest.raises(GCSObjectURIError): + file_handler._parse_object_uri(file_path) + + @responses.activate + def test_get_file_staging_location(self, file_handler: GCSFileHandler): + test_staging_location = "gs://staging/test/file_id" + responses.add(responses.POST, f"{self.BASE_TEST_URL}/getFileLocation", + json={"Location": test_staging_location}, status=http.HTTPStatus.OK) + + assert test_staging_location == file_handler.get_file_staging_location("/test/file_id") + assert responses.calls[0].request.body == json.dumps({"FileID": "file_id"}) + + @pytest.mark.parametrize( + "http_status, reason", + [ + pytest.param(http.HTTPStatus.NOT_FOUND, "Not Found."), + ] + ) + @responses.activate + def test_get_file_staging_location_error(self, file_handler: GCSFileHandler, + http_status: str, reason: str): + responses.add(responses.POST, f"{self.BASE_TEST_URL}/getFileLocation", + status=http_status, body=reason) + + with pytest.raises((tenacity.RetryError, requests.exceptions.HTTPError)): + file_handler.get_file_staging_location("/test/file_id") + + @responses.activate + def test_get_file_permanent_location(self, file_handler: GCSFileHandler): + test_record_id = "test_record_id" + test_permanent_location = "gs://permanent/test/file_id" + json_response = { + "signedUrl": "test_signed", + "unsignedUrl": test_permanent_location, + "kind": "test_kind" + } + responses.add(responses.GET, f"{self.BASE_TEST_URL}/v1/files/{test_record_id}/downloadURL", + json=json_response, status=http.HTTPStatus.OK) + + assert test_permanent_location == file_handler.get_file_permanent_location(test_record_id) + + @pytest.mark.parametrize( + "http_status, reason", + [ + pytest.param(http.HTTPStatus.NOT_FOUND, "Not Found."), + ] + ) + @responses.activate + def test_get_file_permanent_location_error(self, file_handler: GCSFileHandler, + http_status: str, reason: str): + test_record_id = "test_record_id" + responses.add(responses.GET, f"{self.BASE_TEST_URL}/v1/files/{test_record_id}/downloadURL", + status=http_status, body=reason) + + with pytest.raises((tenacity.RetryError, requests.exceptions.HTTPError)): + file_handler.get_file_permanent_location(test_record_id) + + @pytest.mark.parametrize( + "wp_records_file_path", + [ + RECORD_SEISMIC_TRACE_DATA_VALID_PATH, + ] + ) + @responses.activate + def test_save_file_record(self, file_handler: GCSFileHandler, wp_records_file_path: str): + with open(wp_records_file_path) as cf: + file_record, unused_wpc_record, unused_wp_record = json.load(cf) + + test_record_id = "test_record_id" + responses.add(responses.POST, f"{self.BASE_TEST_URL}/v1/files/metadata", + json={"id": test_record_id}, status=http.HTTPStatus.OK) + + assert test_record_id == file_handler.save_file_record(file_record) + + @pytest.mark.parametrize( + "wp_records_file_path", + [ + RECORD_SEISMIC_TRACE_DATA_VALID_PATH, + ] + ) + def test_save_file_record_raises(self, file_handler: GCSFileHandler, wp_records_file_path: str): + with open(wp_records_file_path) as cf: + file_record, unused_wpc_record, unused_wp_record = json.load(cf) + + file_record["data"]["DatasetProperties"]["FileSourceInfo"].pop("FileSource") + + with pytest.raises(InvalidFileRecordData): + file_handler.save_file_record(file_record) diff --git a/tests/plugin-unit-tests/test_file_upload.py b/tests/plugin-unit-tests/test_file_upload.py deleted file mode 100644 index 246d768ccf1d7f0f38182629e40fda8a3dc5b8c4..0000000000000000000000000000000000000000 --- a/tests/plugin-unit-tests/test_file_upload.py +++ /dev/null @@ -1,66 +0,0 @@ -# Copyright 2020 Google LLC -# Copyright 2020 EPAM Systems -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import io -import os -import sys - -sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") -sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") - -from libs.exceptions import GCSObjectURIError -import pytest -from libs.context import Context -from libs.refresh_token import AirflowTokenRefresher -from libs.upload_file import GCSFileUploader - - -class TestSourceFileChecker: - - @pytest.fixture() - def file_uploader(self, monkeypatch): - context = Context(data_partition_id="test", app_key="") - file_uploader = GCSFileUploader("http://test", AirflowTokenRefresher(), - context) - monkeypatch.setattr(file_uploader, "_get_signed_url_request", - lambda *args, **kwargs: ("test", "test")) - monkeypatch.setattr(file_uploader, "_upload_file_request", - lambda *args, **kwargs: None) - monkeypatch.setattr(file_uploader, "_get_file_location_request", - lambda *args, **kwargs: "test") - return file_uploader - - def test_get_file_from_bucket( - self, - monkeypatch, - file_uploader: GCSFileUploader - ): - file = io.RawIOBase() - monkeypatch.setattr(file_uploader, "get_file_from_bucket", - lambda *args, **kwargs: (file, "test")) - file_uploader.upload_file("gs://test/test") - - @pytest.mark.parametrize( - "file_path", - [ - pytest.param("gs://test"), - pytest.param("://test"), - pytest.param("test"), - ] - ) - def test_invalid_gcs_object_uri(self, file_uploader: GCSFileUploader, - file_path: str): - with pytest.raises(GCSObjectURIError): - file_uploader._parse_object_uri(file_path) diff --git a/tests/plugin-unit-tests/test_manifest_processor_r3.py b/tests/plugin-unit-tests/test_manifest_processor_r3.py index 589c0e8b84fe62e199c40b42a2be6261f3df8f42..cc8d1ba3ce65fbc73948931b98fc99126ba9b649 100644 --- a/tests/plugin-unit-tests/test_manifest_processor_r3.py +++ b/tests/plugin-unit-tests/test_manifest_processor_r3.py @@ -23,7 +23,7 @@ sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") from libs.context import Context -from libs.upload_file import GCSFileUploader +from libs.handle_file import GCSFileHandler from libs.source_file_check import GCSSourceFileChecker from libs.refresh_token import AirflowTokenRefresher from libs.exceptions import EmptyManifestError @@ -37,7 +37,7 @@ from file_paths import ( MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, RECORD_WELLBORE_VALID_PATH, RECORD_SEISMIC_TRACE_DATA_VALID_PATH, -) + TRAVERSAL_WELLBORE_VALID_PATH, TRAVERSAL_MANIFEST_EMPTY_PATH, TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH) from libs import process_manifest_r3 TENANT = "opendes" @@ -77,24 +77,36 @@ class TestManifestProcessor: monkeypatch.setattr(requests, "put", mockresponse) @pytest.fixture(autouse=True) - def manifest_processor(self, monkeypatch, conf_path: str): + def manifest_processor(self, monkeypatch, conf_path: str, traversal_manifest_file: str): with open(conf_path) as f: conf = json.load(f) + with open(traversal_manifest_file) as f: + manifest_file = json.load(f) + manifest_records = manifest_file context = Context.populate(conf) token_refresher = AirflowTokenRefresher() - file_uploader = GCSFileUploader("test", token_refresher, context) + file_handler = GCSFileHandler("test", token_refresher, context) + source_file_checker = GCSSourceFileChecker() manifest_processor = process_manifest_r3.ManifestProcessor( storage_url="", - dagrun_conf=conf, + manifest_records=manifest_records, token_refresher=token_refresher, context=context, - file_uploader = file_uploader, - source_file_checker=GCSSourceFileChecker(), + file_handler=file_handler, + source_file_checker=source_file_checker, ) monkeypatch.setattr(manifest_processor, "generate_id", lambda manifest: "test_id") - monkeypatch.setattr(file_uploader, "upload_file", - lambda *args, **kwargs: "test") + monkeypatch.setattr(file_handler, "upload_file", + lambda *args, **kwargs: "/test/source_file") + monkeypatch.setattr(file_handler, "save_file_record", + lambda *args, **kwars: "test_file_record_id") + monkeypatch.setattr(file_handler, "get_file_staging_location", + lambda *args, **kwargs: "gs://staging/test/source_file") + monkeypatch.setattr(file_handler, "get_file_permanent_location", + lambda *args, **kwargs: "gs://permanent/test/source_file") + monkeypatch.setattr(source_file_checker, "does_file_exist", + lambda *args, **kwargs: None) return manifest_processor @pytest.fixture() @@ -106,35 +118,11 @@ class TestManifestProcessor: return json.load(f) @pytest.mark.parametrize( - "conf_path,records_file_path", - [ - pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, - RECORD_SEISMIC_TRACE_DATA_VALID_PATH, - id="Valid WorkProduct"), - pytest.param(MANIFEST_WELLBORE_VALID_PATH, - RECORD_WELLBORE_VALID_PATH, - id="Valid Wellbore"), - ] - ) - def test_create_manifest_records( - self, - manifest_processor: process_manifest_r3.ManifestProcessor, - mock_records_list: list, - conf_path: str, - records_file_path: str - ): - actual_records = manifest_processor.create_manifest_records() - sort_predicate = lambda x: x["kind"] - actual_records.sort(key=sort_predicate) - mock_records_list.sort(key=sort_predicate) - assert actual_records == mock_records_list, DeepDiff(actual_records, mock_records_list, - ignore_order=True) - - @pytest.mark.parametrize( - "conf_path,records_file_path", + "conf_path,traversal_manifest_file,records_file_path", [ pytest.param( MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, RECORD_SEISMIC_TRACE_DATA_VALID_PATH, id="Valid WorkProduct" ), @@ -145,71 +133,82 @@ class TestManifestProcessor: monkeypatch, manifest_processor: process_manifest_r3.ManifestProcessor, mock_records_list: list, + traversal_manifest_file: str, conf_path: str, records_file_path: str ): self.monkeypatch_storage_response(monkeypatch) - manifest_processor.save_record({}, records_file_path) + manifest_processor.save_record_to_storage({}, records_file_path) @pytest.mark.parametrize( - "conf_path", + "conf_path,traversal_manifest_file", [ - MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH + pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH) ] ) def test_save_record_invalid_storage_response_value( self, monkeypatch, manifest_processor: process_manifest_r3.ManifestProcessor, + traversal_manifest_file: str, conf_path: str ): self.monkeypatch_storage_response(monkeypatch, b"{}") with pytest.raises(ValueError): - manifest_processor.save_record({}, [{}]) + manifest_processor.save_record_to_storage({}, [{}]) @pytest.mark.parametrize( - "conf_path", + "conf_path,traversal_manifest_file", [ - pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, id="Valid WorkProduct") + pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, + id="Valid WorkProduct") ] ) def test_save_record_storage_response_http_error( self, monkeypatch, manifest_processor: process_manifest_r3.ManifestProcessor, + traversal_manifest_file: str, conf_path: str ): self.monkeypatch_storage_response_error(monkeypatch, http.HTTPStatus.INTERNAL_SERVER_ERROR) with pytest.raises(requests.HTTPError): - manifest_processor.save_record({}, conf_path) + manifest_processor.save_record_to_storage({}, conf_path) @pytest.mark.parametrize( - "conf_path", + "conf_path,traversal_manifest_file", [ - pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, id="WorkProduct"), - pytest.param(MANIFEST_WELLBORE_VALID_PATH, id="Master"), - + pytest.param(MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, + id="WorkProduct"), + pytest.param(MANIFEST_WELLBORE_VALID_PATH, + TRAVERSAL_WELLBORE_VALID_PATH, + id="Master"), ] ) def test_process_manifest_valid( self, monkeypatch, manifest_processor: process_manifest_r3.ManifestProcessor, + traversal_manifest_file: str, conf_path: str ): self.monkeypatch_storage_response(monkeypatch) manifest_processor.process_manifest() @pytest.mark.parametrize( - "conf_path", + "conf_path,traversal_manifest_file", [ - pytest.param(MANIFEST_EMPTY_PATH, id="Empty Manifest"), + pytest.param(MANIFEST_EMPTY_PATH, TRAVERSAL_MANIFEST_EMPTY_PATH, id="Empty Manifest"), ] ) def test_process_empty_manifest( self, monkeypatch, manifest_processor: process_manifest_r3.ManifestProcessor, + traversal_manifest_file: str, conf_path: str ): self.monkeypatch_storage_response(monkeypatch) @@ -217,9 +216,9 @@ class TestManifestProcessor: manifest_processor.process_manifest() @pytest.mark.parametrize( - "conf_path,expected_kind_name", + "conf_path,expected_kind_name,traversal_manifest_file", [ - pytest.param(MANIFEST_WELLBORE_VALID_PATH, "Wellbore", id="Valid Wellbore"), + pytest.param(MANIFEST_WELLBORE_VALID_PATH, "TestMaster", TRAVERSAL_WELLBORE_VALID_PATH, id="Valid Wellbore"), ] ) def test_get_kind( @@ -227,37 +226,42 @@ class TestManifestProcessor: monkeypatch, manifest_processor: process_manifest_r3.ManifestProcessor, conf_path: str, + traversal_manifest_file: str, expected_kind_name: str ): - for manifest_part in manifest_processor.data_object["manifest"]: - kind = manifest_part["kind"] + for manifest_part in manifest_processor.manifest_records: + kind = manifest_part["entity"]["kind"] assert expected_kind_name == manifest_processor._get_kind_name(kind) @pytest.mark.parametrize( - "conf_path,expected_kind_name", + "conf_path,expected_kind_name,traversal_manifest_file", [ - pytest.param(MANIFEST_WELLBORE_VALID_PATH, "Wellbore", id="Valid Wellbore"), + pytest.param(MANIFEST_WELLBORE_VALID_PATH, "Wellbore", TRAVERSAL_WELLBORE_VALID_PATH, id="Valid Wellbore"), ] ) def test_generate_id( self, monkeypatch, conf_path: str, + traversal_manifest_file: str, expected_kind_name: str ): with open(conf_path) as f: conf = json.load(f) + with open(traversal_manifest_file) as f: + manifest_file = json.load(f) + manifest_records = manifest_file context = process_manifest_r3.Context.populate(conf) manifest_processor = process_manifest_r3.ManifestProcessor( storage_url="", - dagrun_conf=conf, + manifest_records=manifest_records, token_refresher=AirflowTokenRefresher(), context=context, - file_uploader=GCSFileUploader("test", AirflowTokenRefresher(), context), + file_handler=GCSFileHandler("test", AirflowTokenRefresher(), context), source_file_checker=GCSSourceFileChecker() ) - for manifest_part in manifest_processor.data_object["manifest"]: - group_type = manifest_part["groupType"] - kind_name = manifest_processor._get_kind_name(manifest_part["kind"]) - generated_id = manifest_processor.generate_id(manifest_part) + for manifest_part in manifest_processor.manifest_records: + group_type = manifest_part["entity"]["groupType"] + kind_name = manifest_processor._get_kind_name(manifest_part["entity"]["kind"]) + generated_id = manifest_processor.generate_id(manifest_part["entity"]) assert generated_id.startswith(f"{TENANT}:{group_type}_{kind_name}:") diff --git a/tests/plugin-unit-tests/test_manifest_traversal.py b/tests/plugin-unit-tests/test_manifest_traversal.py new file mode 100644 index 0000000000000000000000000000000000000000..0dd5653c93a09a8f8509f3cbb7881a85ed032c29 --- /dev/null +++ b/tests/plugin-unit-tests/test_manifest_traversal.py @@ -0,0 +1,82 @@ +# Copyright 2020 Google LLC +# Copyright 2020 EPAM Systems +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import json +import os +import sys + +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins") +sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags") + +import pytest +from file_paths import MANIFEST_WELLBORE_VALID_PATH, TRAVERSAL_WELLBORE_VALID_PATH, \ + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, MANIFEST_EMPTY_PATH, \ + TRAVERSAL_MANIFEST_EMPTY_PATH, MANIFEST_GENERIC_SCHEMA_PATH +from libs.exceptions import EmptyManifestError +from libs.traverse_manifest import ManifestTraversal + + +class TestManifestTraversal: + @pytest.fixture + def manifest_traversal(self, monkeypatch, manifest_file: str, manifest_schema_file: str): + with open(manifest_file) as f: + conf_manifest_file = json.load(f) + with open(manifest_schema_file) as f: + manifest_schema = json.load(f) + traversal = ManifestTraversal( + conf_manifest_file, + manifest_schema + ) + return traversal + + @pytest.mark.parametrize( + "manifest_file,manifest_schema_file,traversal_manifest_file", + [ + pytest.param( + MANIFEST_WELLBORE_VALID_PATH, + MANIFEST_GENERIC_SCHEMA_PATH, + TRAVERSAL_WELLBORE_VALID_PATH, + id="Valid manifest Wellore"), + pytest.param( + MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, + MANIFEST_GENERIC_SCHEMA_PATH, + TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, + id="Valid manifest WPC"), + ] + ) + def test_traversal_manifest(self, monkeypatch, manifest_traversal, manifest_file: str, + manifest_schema_file: str, traversal_manifest_file: str): + with open(traversal_manifest_file) as f: + traversal_manifest = json.load(f) + manifest_records = manifest_traversal.traverse_manifest() + assert manifest_records == traversal_manifest + + @pytest.mark.parametrize( + "manifest_file,manifest_schema_file,traversal_manifest_file", + [ + pytest.param( + MANIFEST_EMPTY_PATH, + MANIFEST_GENERIC_SCHEMA_PATH, + TRAVERSAL_MANIFEST_EMPTY_PATH, + id="Empty manifest"), + ] + ) + def test_traversal_empty_manifest(self, monkeypatch, + manifest_traversal, + manifest_file: str, + manifest_schema_file: str, + traversal_manifest_file: str): + with pytest.raises(EmptyManifestError): + manifest_traversal.traverse_manifest() diff --git a/tests/plugin-unit-tests/test_operators_r3.py b/tests/plugin-unit-tests/test_operators_r3.py index 2b34ce61960ac65288a6fdab594ac13d1211eb60..f99441d3801b9d964216cd0cfd85596df56a5d9b 100644 --- a/tests/plugin-unit-tests/test_operators_r3.py +++ b/tests/plugin-unit-tests/test_operators_r3.py @@ -32,14 +32,12 @@ import requests from file_paths import ( MANIFEST_WELLBORE_VALID_PATH, - SEARCH_VALID_RESPONSE_PATH -) + SEARCH_VALID_RESPONSE_PATH, MANIFEST_GENERIC_SCHEMA_PATH) from operators.process_manifest_r3 import ProcessManifestOperatorR3, SchemaValidator, \ ManifestProcessor from operators.search_record_id import SearchRecordIdOperator from operators.update_status import UpdateStatusOperator -from libs.upload_file import GCSFileUploader -from libs.refresh_token import AirflowTokenRefresher +from libs.handle_file import GCSFileHandler from mock_responses import MockSearchResponse, MockWorkflowResponse CustomOperator = TypeVar("CustomOperator") @@ -70,10 +68,16 @@ class TestOperators(object): return task, context def test_process_manifest_r3_operator(self, monkeypatch): - monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj: None) - monkeypatch.setattr(ManifestProcessor, "save_record", + + 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_manifest", lambda obj, entities: entities) + monkeypatch.setattr(ManifestProcessor, "save_record_to_storage", lambda obj, headers, request_data: MockStorageResponse()) - monkeypatch.setattr(GCSFileUploader, "upload_file", + monkeypatch.setattr(GCSFileHandler, "upload_file", lambda *args, **kwargs: "test") task, context = self._create_task(ProcessManifestOperatorR3) diff --git a/tests/plugin-unit-tests/test_schema_validator_r3.py b/tests/plugin-unit-tests/test_schema_validator_r3.py index bae115d64e9f77b5e10b1f4e9a6e2f35531f9b12..1b138b4cf63a1c0cd7a5a92a0f15c7fa5b831312 100644 --- a/tests/plugin-unit-tests/test_schema_validator_r3.py +++ b/tests/plugin-unit-tests/test_schema_validator_r3.py @@ -28,16 +28,23 @@ from file_paths import ( DATA_PATH_PREFIX, MANIFEST_EMPTY_PATH, SCHEMA_FILE_VALID_PATH, + SCHEMA_GENERIC_MASTERDATA_PATH, SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH, SCHEMA_WORK_PRODUCT_VALID_PATH, + SCHEMA_TEST_MASTERDATA_PATH, MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, MANIFEST_WELLBORE_VALID_PATH, - SCHEMA_WELLBORE_VALID_PATH + MANIFEST_GENERIC_PATH, + MANIFEST_GENERIC_SCHEMA_PATH, + SCHEMA_WELLBORE_VALID_PATH, + TRAVERSAL_WELLBORE_VALID_PATH, + TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, + TRAVERSAL_MANIFEST_EMPTY_PATH ) from mock_responses import MockSchemaResponse from libs.context import Context from libs.refresh_token import AirflowTokenRefresher -from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError +from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError import pytest from libs.validate_schema import SchemaValidator @@ -48,14 +55,14 @@ TENANT = "opendes" class TestSchemaValidator: @pytest.fixture - def schema_validator(self, monkeypatch, manifest_file: str, schema_file: str): - with open(manifest_file) as f: - manifest_file = json.load(f) - conf = copy.deepcopy(manifest_file) - context = Context.populate(conf) + def schema_validator( + self, + monkeypatch, + schema_file: str + ): + context = Context(app_key="", data_partition_id="") validator = SchemaValidator( "", - conf, AirflowTokenRefresher(), context ) @@ -64,60 +71,53 @@ class TestSchemaValidator: lambda *args, **kwargs: MockSchemaResponse(schema_file)) return validator - @pytest.mark.parametrize( - "manifest_file,schema_file", - [ - pytest.param( - MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, None, - id="Valid manifest_file") - ] - ) - def test_schema_validator_validate_workproduct_manifest(self, - monkeypatch, - schema_validator: SchemaValidator, - manifest_file: str, - schema_file: str): - def mock_get_schema(uri: str): - if ":WorkProduct:" in uri: - schema_path = SCHEMA_WORK_PRODUCT_VALID_PATH - elif ":SeismicTraceData:" in uri: - schema_path = SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH - elif ":File:" in uri: - schema_path = SCHEMA_FILE_VALID_PATH - else: - print(uri) - raise - with open(schema_path) as f: - schema_content = json.load(f) - return schema_content - - monkeypatch.setattr(schema_validator, "get_schema_request", mock_get_schema) - schema_validator.validate_manifest() + @staticmethod + def mock_get_schema(uri: str): + if "WorkProduct" in uri: + schema_path = SCHEMA_WORK_PRODUCT_VALID_PATH + elif "SeismicTraceData" in uri: + schema_path = SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH + elif "File" in uri: + schema_path = SCHEMA_FILE_VALID_PATH + elif "GenericMasterData" in uri: + schema_path = SCHEMA_GENERIC_MASTERDATA_PATH + elif "TestMaster" in uri: + schema_path = SCHEMA_TEST_MASTERDATA_PATH + else: + print(uri) + raise Exception(f"Can't get such a schema {uri} in data files of tests") + with open(schema_path) as f: + schema_content = json.load(f) + return schema_content @pytest.mark.parametrize( - "manifest_file,schema_file", + "traversal_manifest_file_path,schema_file", [ pytest.param( - MANIFEST_WELLBORE_VALID_PATH, - SCHEMA_WELLBORE_VALID_PATH, - id="Valid manifest Wellore"), + TRAVERSAL_WELLBORE_VALID_PATH, + None, + id="Valid manifest_file" + ) ] ) - def test_schema_validator_master_manifest(self, monkeypatch, schema_validator, manifest_file, - schema_file): - def mock_get_schema(uri: str): - with open(schema_file) as f: - schema_content = json.load(f) - return schema_content - - monkeypatch.setattr(schema_validator, "get_schema_request", mock_get_schema) - schema_validator.validate_manifest() + def test_schema_validator_master_manifest( + self, + monkeypatch, + schema_validator, + traversal_manifest_file_path: str, + schema_file + ): + monkeypatch.setattr(schema_validator, "get_schema", self.mock_get_schema) + with open(traversal_manifest_file_path) as f: + manifest_file = json.load(f) + validated_records = schema_validator.validate_manifest(manifest_file) + assert len(manifest_file) == len(validated_records) @pytest.mark.parametrize( - "manifest_file,schema_file", + "manifest_file,traversal_manifest_file,schema_file", [ pytest.param( - MANIFEST_EMPTY_PATH, None, + MANIFEST_EMPTY_PATH, TRAVERSAL_MANIFEST_EMPTY_PATH, None, id="Empty Manifest"), ] ) @@ -125,31 +125,38 @@ class TestSchemaValidator: 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() + schema_validator.validate_manifest(manifest_file) @pytest.mark.parametrize( - "manifest_file,schema_file", + "traversal_manifest_file,schema_file", [ pytest.param( - f"{DATA_PATH_PREFIX}/invalid/NotOsduFormat.json", None, + f"{DATA_PATH_PREFIX}/invalid/TraversalNotOSDUFormatManifest.json", + None, id="Not OSDU FORMAT"), ] ) def test_schema_validator_not_osdu_format(self, monkeypatch, schema_validator: SchemaValidator, - manifest_file: str, + traversal_manifest_file: str, schema_file: str): - with pytest.raises(NotOSDUShemaFormatError): - schema_validator.validate_manifest() + with open(traversal_manifest_file) as f: + manifest_file = json.load(f) + with pytest.raises(NotOSDUSchemaFormatError): + schema_validator.validate_manifest(manifest_file) @pytest.mark.parametrize( - "manifest_file,schema_file,kind", + "manifest_file,traversal_manifest_file,schema_file,kind", [ pytest.param( MANIFEST_WELLBORE_VALID_PATH, + TRAVERSAL_WELLBORE_VALID_PATH, SCHEMA_WELLBORE_VALID_PATH, "opendes:osdu:Wellbore:0.3.0", id="Valid manifest Wellore"), @@ -158,15 +165,17 @@ class TestSchemaValidator: def test_get_schema_request(self, schema_validator: SchemaValidator, manifest_file: str, + traversal_manifest_file: str, schema_file: str, kind: str): schema_validator.get_schema_request(kind) @pytest.mark.parametrize( - "manifest_file,schema_file,kind", + "manifest_file,traversal_manifest_file,schema_file,kind", [ pytest.param( MANIFEST_WELLBORE_VALID_PATH, + TRAVERSAL_WELLBORE_VALID_PATH, SCHEMA_WELLBORE_VALID_PATH, "opendes:osdu:Wellbore:0.3.0", id="Valid manifest Wellore"), @@ -176,6 +185,7 @@ class TestSchemaValidator: monkeypatch, schema_validator: SchemaValidator, manifest_file: str, + traversal_manifest_file: str, schema_file: str, kind: str): monkeypatch.setattr(requests, @@ -184,3 +194,29 @@ class TestSchemaValidator: http.HTTPStatus.INTERNAL_SERVER_ERROR)) with pytest.raises(requests.HTTPError): schema_validator.get_schema(kind) + + @pytest.mark.parametrize( + "manifest_file,schema_file", + [ + pytest.param( + MANIFEST_GENERIC_PATH, + MANIFEST_GENERIC_SCHEMA_PATH, + id="Valid generic manifest"), + ] + ) + def test_generic_manifest_validation( + self, + monkeypatch, + schema_validator: SchemaValidator, + manifest_file: str, + schema_file + ): + def mock_get_schema(uri: str): + with open(schema_file) as f: + schema_content = json.load(f) + return schema_content + + 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}) diff --git a/tests/unit_tests.sh b/tests/unit_tests.sh index ae3c686de750d7e09b2df1eda8042ba6cda29cc9..c93dd890b2bf77994ce0f7b5aa4c9fd3ae11e648 100644 --- a/tests/unit_tests.sh +++ b/tests/unit_tests.sh @@ -1,5 +1,6 @@ pip uninstall enum34 -y pip install pytest +pip install responses pip install --upgrade google-api-python-client chmod +x tests/set_airflow_env.sh export AIRFLOW_SRC_DIR="/usr/local/airflow/"