diff --git a/scripts/convert_R2_schemas_to_R3.py b/scripts/convert_R2_schemas_to_R3.py
deleted file mode 100644
index fa8572f56e701992053d834e3b8c3147604e6ec9..0000000000000000000000000000000000000000
--- a/scripts/convert_R2_schemas_to_R3.py
+++ /dev/null
@@ -1,81 +0,0 @@
-import glob
-import json
-import os
-import re
-
-from collections import UserString
-
-TENANT = "opendes"
-AUTHORITY = "osdu"
-SCHEMAS_DIR = os.environ["SCHEMAS_DIR"]
-
-
-class JsonString(UserString):
-    REF_REGEXP = r"(?P<abstract_perfix>\.\.\/abstract/)(?P<kind_name>\w+)\.(?P<version>\d+\.\d+\.\d+)\.json"
-    NAMESPACE_REGEXP = r"\<namespace\>"
-
-    def repl_closure(self, match: re.match):
-        if not match.groups:
-            print(self.data)
-            raise Exception
-        kind_name = match.group('kind_name')
-        version = match.group('version')
-        repl = f"{TENANT}:{AUTHORITY}:{kind_name}:{version}"
-        return repl
-
-    def replace_refs(self):
-        self.data = re.sub(self.REF_REGEXP, self.repl_closure, self.data)
-        return self
-
-    def replace_namespaces(self):
-        self.data = re.sub(self.NAMESPACE_REGEXP, TENANT, self.data)
-        return self
-
-    @staticmethod
-    def lower_first_letter(val: str):
-        if val[0].islower():
-            pass
-        elif val in (
-            "ACL",
-            "Legals",
-            "ID"
-        ):
-            val = val.lower()
-        else:
-            val = val.replace(val[0], val[0].lower(), 1)
-        return val
-
-    def to_pascal_case(self):
-        tmp_properties = {}
-        tmp_required = []
-        json_file_dict = json.loads(self.data)
-        try:
-            if "schemaInfo" in json_file_dict:  # if schema has additional fields to be recorded
-                content = json_file_dict["schema"]
-            else:
-                content = json_file_dict
-            if "properties" in content:
-                for key, value in content["properties"].items():
-                    tmp_properties[self.lower_first_letter(key)] = value
-                content["properties"] = tmp_properties
-            if "required" in content:
-                for i in content["required"]:
-                    tmp_required.append(self.lower_first_letter(i))
-                content["required"] = tmp_required
-            self.data = json.dumps(json_file_dict, indent=4)
-            return self
-        except Exception as e:
-            print(self.data)
-            raise e
-
-
-for file_path in glob.glob(SCHEMAS_DIR + "/*.json"):
-    try:
-        with open(file_path, "r") as file:
-            content = file.read()
-            content = JsonString(content).replace_refs().replace_namespaces().to_pascal_case().data
-        with open(file_path, "w") as file:
-            file.write(content)
-    except Exception as e:
-        print(f"Error on file {file_path}")
-        raise e
diff --git a/src/dags/default-ingest.py b/src/dags/default-ingest.py
index 18ff8e01401b451b2f230308279b8d21ab73b6c1..fe4f30779af3a5b245f34120dfd9ffa4d5da25fc 100644
--- a/src/dags/default-ingest.py
+++ b/src/dags/default-ingest.py
@@ -21,7 +21,7 @@ from datetime import timedelta
 import airflow.utils.dates
 from airflow import DAG
 from airflow.operators.python_operator import PythonOperator
-from operators.update_status_op import UpdateStatusOperator
+from operators.update_status import UpdateStatusOperator
 
 from libs.create_records import create_records  # isort:skip
 
diff --git a/src/dags/libs/exceptions.py b/src/dags/libs/exceptions.py
index 43978146e43982a2361ffa99fe3381ff7b698e2a..5042509037e9c02cfc1ec0d6b185dd84014fd56a 100644
--- a/src/dags/libs/exceptions.py
+++ b/src/dags/libs/exceptions.py
@@ -46,3 +46,9 @@ class GetSchemaError(Exception):
     Raise when can't find schema.
     """
     pass
+
+class NotOSDUShemaFormatError(Exception):
+    """
+    Raise when schema doesn't correspond OSDU format
+    """
+    pass
diff --git a/src/dags/osdu-ingest-r2.py b/src/dags/osdu-ingest-r2.py
index 03fb1d94d09d7f12dd935a67b0ac357b0bf421e0..27891485f27679423740656e72689dc4f84e05c0 100644
--- a/src/dags/osdu-ingest-r2.py
+++ b/src/dags/osdu-ingest-r2.py
@@ -19,9 +19,9 @@ from datetime import timedelta
 
 import airflow
 from airflow import DAG
-from operators.process_manifest_r2_op import ProcessManifestOperatorR2
-from operators.search_record_id_op import SearchRecordIdOperator
-from operators.update_status_op import UpdateStatusOperator
+from operators.process_manifest_r2 import ProcessManifestOperatorR2
+from operators.search_record_id import SearchRecordIdOperator
+from operators.update_status import UpdateStatusOperator
 
 default_args = {
     "start_date": airflow.utils.dates.days_ago(0),
diff --git a/src/dags/osdu-ingest-r3.py b/src/dags/osdu-ingest-r3.py
index e0292cae08e789332057c5030375ea05cf1b8d5c..b8a87e1f1ff286fe70c47b419aaff60d784a79a0 100644
--- a/src/dags/osdu-ingest-r3.py
+++ b/src/dags/osdu-ingest-r3.py
@@ -20,8 +20,7 @@ from datetime import timedelta
 import airflow
 from airflow import DAG
 from operators.process_manifest_r3 import ProcessManifestOperatorR3
-from operators.search_record_id_op import SearchRecordIdOperator
-from operators.update_status_op import UpdateStatusOperator
+from operators.update_status import UpdateStatusOperator
 
 default_args = {
     "start_date": airflow.utils.dates.days_ago(0),
@@ -55,12 +54,5 @@ process_manifest_op = ProcessManifestOperatorR3(
     dag=dag
 )
 
-search_record_ids_op = SearchRecordIdOperator(
-    task_id="search_record_ids_task",
-    provide_context=True,
-    dag=dag,
-    retries=4
-)
-
 update_status_running_op >> process_manifest_op >> \
-search_record_ids_op >> update_status_finished_op # pylint: disable=pointless-statement
+update_status_finished_op # pylint: disable=pointless-statement
diff --git a/src/plugins/operators/process_manifest_r2_op.py b/src/plugins/operators/process_manifest_r2.py
similarity index 96%
rename from src/plugins/operators/process_manifest_r2_op.py
rename to src/plugins/operators/process_manifest_r2.py
index 89ddc08976fccef1d0b44ef4cd8e03a23698b287..2641c9a236e5ee259ceedb44e10404942dce8b0a 100644
--- a/src/plugins/operators/process_manifest_r2_op.py
+++ b/src/plugins/operators/process_manifest_r2.py
@@ -22,7 +22,6 @@ import sys
 import time
 import uuid
 from collections import Counter
-from datetime import datetime
 from typing import Tuple
 from urllib.error import HTTPError
 
@@ -51,15 +50,6 @@ logger = logging.getLogger("Dataload")
 logger.setLevel(logging.INFO)
 logger.addHandler(handler)
 
-timestamp = datetime.now().isoformat()
-
-# Set up file logger
-handler = logging.FileHandler(f"execution_{timestamp}.log")
-handler.setFormatter(logging.Formatter("%(message)s"))
-file_logger = logging.getLogger("Execution")
-file_logger.setLevel(logging.INFO)
-file_logger.addHandler(handler)
-
 # Some constants, used by script
 SEARCH_OK_RESPONSE_CODES = [200]
 DATA_LOAD_OK_RESPONSE_CODES = [201]
@@ -288,7 +278,7 @@ def send_request(headers, request_data):
                                     headers=headers)
 
             if response.status_code in DATA_LOAD_OK_RESPONSE_CODES:
-                file_logger.info(",".join(map(str, response.json()["recordIds"])))
+                logger.info(",".join(map(str, response.json()["recordIds"])))
                 return response
 
             reason = response.text[:250]
diff --git a/src/plugins/operators/process_manifest_r3.py b/src/plugins/operators/process_manifest_r3.py
index b0323cd7b78e987363267aafc38b78359064bcdd..feb2063bdc039f9dbcdc9bf0d1ec8f04a76d29db 100644
--- a/src/plugins/operators/process_manifest_r3.py
+++ b/src/plugins/operators/process_manifest_r3.py
@@ -20,12 +20,13 @@ import logging
 import sys
 import uuid
 from datetime import datetime
+from typing import List
 
 import jsonschema
 import requests
 import tenacity
 from airflow.models import BaseOperator, Variable
-from libs.exceptions import EmptyManifestError, GetSchemaError
+from libs.exceptions import EmptyManifestError, NotOSDUShemaFormatError
 from libs.refresh_token import refresh_token
 
 # Set up base logger
@@ -36,21 +37,16 @@ logger = logging.getLogger("Dataload")
 logger.setLevel(logging.INFO)
 logger.addHandler(handler)
 
-timestamp = datetime.now().isoformat()
-
-# Set up file logger
-handler = logging.FileHandler(f"execution_{timestamp}.log")
-handler.setFormatter(logging.Formatter("%(message)s"))
-file_logger = logging.getLogger("Execution")
-file_logger.setLevel(logging.INFO)
-file_logger.addHandler(handler)
-
 RETRIES = 3
 TIMEOUT = 1
 
 
 @dataclasses.dataclass
 class Context(object):
+    """
+    Store data-partition-id and AppKey passed via Payload field of dagrun.conf.
+    Remove Payload from dagrun.conf.
+    """
     data_partition_id: str
     app_key: str
 
@@ -65,11 +61,11 @@ class Context(object):
 
 class OSDURefResolver(jsonschema.RefResolver):
 
-    def __init__(self, schema_service, *args, **kwargs):
+    def __init__(self, schema_service: str, *args, **kwargs):
         super(OSDURefResolver, self).__init__(*args, **kwargs)
         self.schema_service = schema_service
 
-    def resolve_fragment(self, document, fragment):
+    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
@@ -88,7 +84,7 @@ class OSDURefResolver(jsonschema.RefResolver):
 class SchemaValidator(object):
     """Class to validate schema of Manifests."""
 
-    def __init__(self, schema_service, dagrun_conf, context):
+    def __init__(self, schema_service: str, dagrun_conf: dict, context: Context):
         self.schema_service = schema_service
         self.data_object = copy.deepcopy(dagrun_conf)
         self.context = context
@@ -105,38 +101,73 @@ class SchemaValidator(object):
         }
 
     @refresh_token
-    def _get_schema_request(self, headers, uri):
+    def _get_schema_request(self, headers: dict, uri: str) -> requests.Response:
+        """
+        Request to Schema service to retrieve schema.
+        """
         response = requests.get(uri, headers=headers, timeout=60)
         return response
 
-    def get_schema_request(self, uri):
+    def get_schema_request(self, uri: str) -> dict:
+        """
+        Get Schema from Schema service. Change $id field to url of getting schema.
+        """
         if uri.startswith("osdu") or uri.startswith(self.context.data_partition_id):
             uri = f"{self.schema_service}/{uri}"
         response = self._get_schema_request(self.request_headers, uri).json()
         response["$id"] = uri
         return response
 
-    def get_schema(self, kind):
+    def get_schema(self, kind: str) -> dict:
         manifest_schema_uri = f"{self.schema_service}/{kind}"
-        response = self.get_schema_request(manifest_schema_uri)
+        try:
+            response = self.get_schema_request(manifest_schema_uri)
+        except Exception as e:
+            logger.error(f"Error on getting schema of kind '{kind}'")
+            raise e
         return response
 
-    def _validate_schema(self, schema, manifest):
+    def _validate_schema(self, manifest: dict, schema: dict = None):
+        """
+        Validate schema. If argument schema is not defined, then use schema service to retrieve
+        corresponding schema.
+        """
+        if not schema:
+            schema = self.get_schema(manifest["kind"])
+        logger.info(f"Validating kind {manifest['kind']}")
         resolver = OSDURefResolver(schema_service=self.schema_service,
                                    base_uri=schema.get("$id", ""), referrer=schema,
                                    handlers=self.resolver_handlers, cache_remote=True)
         validator = jsonschema.Draft7Validator(schema=schema, resolver=resolver)
         validator.validate(manifest)
 
+    def validate_work_product(self, work_product: dict):
+        """
+        Validate WP manifest. Raise error if manifest is not valid.
+        """
+        for key, value in work_product.items():
+            if key != "WorkProduct":
+                for component in value:
+                    self._validate_schema(component)
+            else:
+                self._validate_schema(value)
+
     def validate_manifest(self):
-        for m in self.data_object["manifest"]:
-            manifest_schema = self.get_schema(m["kind"])
-            self._validate_schema(manifest_schema, m)
+        """
+        Validate manifest. Raise error if manifest is not valid.
+        """
+        for manifest in self.data_object["manifest"]:
+            if isinstance(manifest, dict) and manifest.get("kind"):
+                self._validate_schema(manifest)
+            elif manifest.get("WorkProductComponents"):
+                self.validate_work_product(manifest)
+            else:
+                raise NotOSDUShemaFormatError(f"Not valid schema {manifest}")
 
 
 class ManifestProcessor(object):
     """Class to process WP, Master and Reference data"""
-    RECORD_TEMPLATE: dict = {
+    RECORD_TEMPLATE = {
         "legal": {},
         "acl": {},
         "kind": "",
@@ -150,11 +181,26 @@ class ManifestProcessor(object):
         self.data_object = copy.deepcopy(dagrun_conf)
         self.context = context
 
-    def generate_id(self):
-        return f"{self.context.data_partition_id}:doc:{str(uuid.uuid4())}"
+    @staticmethod
+    def _get_kind_name(kind: str) -> str:
+        """
+        osdu:osdu:Well:1.0.0 -> Well
+        """
+        kind_name = kind.split(":")[2]
+        return kind_name
+
+    def generate_id(self, manifest_fragment: dict) -> str:
+        """
+        Generate id to use it in Storage.
+        """
+        group_type = manifest_fragment.get("groupType", "doc")
+        kind = manifest_fragment.get("kind")
+        kind_name = self._get_kind_name(kind)
+        _id = f"{self.context.data_partition_id}:{group_type}_{kind_name}:{str(uuid.uuid4())}"
+        return _id
 
     @property
-    def request_headers(self):
+    def request_headers(self) -> dict:
         headers = {
             'Content-type': 'application/json',
             'data-partition-id': self.context.data_partition_id,
@@ -162,12 +208,12 @@ class ManifestProcessor(object):
         }
         return headers
 
-    def populate_manifest_storage_record(self, manifest):
+    def populate_manifest_storage_record(self, manifest: dict) -> dict:
         """
         Create a record from Master-manifest to store it in Storage service
         """
         record = copy.deepcopy(self.RECORD_TEMPLATE)
-        record["id"] = self.generate_id()
+        record["id"] = self.generate_id(manifest)
         record["kind"] = manifest.pop("kind")
         record["legal"] = manifest.pop("legal")
         record["acl"] = manifest.pop("acl")
@@ -177,13 +223,13 @@ class ManifestProcessor(object):
     @tenacity.retry(tenacity.wait_fixed(TIMEOUT),
                     tenacity.stop_after_attempt(RETRIES))
     @refresh_token
-    def save_record(self, headers, request_data):
+    def save_record(self, headers: dict, request_data: List[dict]) -> requests.Response:
         """
-        Send request to record storage API
+        Send request to record storage API.
         """
         response = requests.put(self.storage_url, json.dumps(request_data), headers=headers)
         if response.ok:
-            file_logger.info(",".join(map(str, response.json()["recordIds"])))
+            logger.info(",".join(map(str, response.json()["recordIds"])))
         else:
             reason = response.text[:250]
             logger.error(f"Request error.")
@@ -191,20 +237,59 @@ class ManifestProcessor(object):
                          f"Response content: {reason}.")
         return response
 
-    def create_manifest_records(self):
+    def process_work_product(self, manifest: dict) -> List[dict]:
+        """
+        Process WP.
+        """
+        wp = manifest["WorkProduct"]
+        records = [self.populate_manifest_storage_record(wp)]
+        return records
+
+    def process_work_product_components(self, manifest: dict) -> List[dict]:
+        """
+        Process list of WPS.
+        """
+        records = []
+        for wpc in manifest["WorkProductComponents"]:
+            record = self.populate_manifest_storage_record(wpc)
+            records.append(record)
+        return records
+
+    def process_work_product_files(self, manifest: dict) -> List[dict]:
+        """
+        Process list of files.
+        """
+        records = []
+        for file in manifest["Files"]:
+            record = self.populate_manifest_storage_record(file)
+            records.append(record)
+        return records
+
+    def process_work_product_manifest(self, manifest: dict) -> List[dict]:
+        file_records = self.process_work_product_files(manifest)
+        wpc_records = self.process_work_product_components(manifest)
+        wp_records = self.process_work_product(manifest)
+        records = file_records + wpc_records + wp_records
+        return records
+
+    def create_manifest_records(self) -> List[dict]:
         manifest_records = []
-        for manifest in self.data_object["manifest"]:
-            record = self.populate_manifest_storage_record(manifest)
-            manifest_records.append(record)
+        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):
+    def process_manifest(self) -> List[str]:
         if "manifest" in self.data_object:
             manifest_records = self.create_manifest_records()
         else:
             raise EmptyManifestError
-        response = self.save_record(self.request_headers,
-                                                  request_data=manifest_records)
+        response = self.save_record(self.request_headers, request_data=manifest_records)
         record_ids = response.json()["recordIds"]
         return record_ids
 
@@ -213,11 +298,11 @@ class ProcessManifestOperatorR3(BaseOperator):
     ui_color = '#dad5ff'
     ui_fgcolor = '#000000'
 
-    def pre_execute(self, context):
+    def pre_execute(self, context: dict):
         self.schema_service_url = Variable.get('schema_service_url')
         self.storage_url = Variable.get('storage_url')
 
-    def execute(self, context):
+    def execute(self, context: dict):
         payload_context = Context.populate(context["dag_run"].conf)
         validator = SchemaValidator(
             self.schema_service_url,
diff --git a/src/plugins/operators/search_record_id_op.py b/src/plugins/operators/search_record_id.py
similarity index 100%
rename from src/plugins/operators/search_record_id_op.py
rename to src/plugins/operators/search_record_id.py
diff --git a/src/plugins/operators/update_status_op.py b/src/plugins/operators/update_status.py
similarity index 100%
rename from src/plugins/operators/update_status_op.py
rename to src/plugins/operators/update_status.py
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
new file mode 100644
index 0000000000000000000000000000000000000000..2aa7957e3c3d907d2df7cf840dd80b074adb55cc
--- /dev/null
+++ b/tests/plugin-unit-tests/data/master/Wellbore.0.3.0.json
@@ -0,0 +1,137 @@
+{
+    "Payload": {
+        "authorization": "Bearer test",
+        "data-partition-id": "opendes",
+        "AppKey": "",
+        "kind_version": "3.0.0"
+    },
+    "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
+    "$filename": "load_Wellbore.1.0.0_350112350400.json",
+    "manifest": [
+        {
+            "id": "srn:opendes:master-data/Wellbore:350112350400",
+            "kind": "opendes:osdu:Wellbore:0.3.0",
+            "groupType": "master-data",
+            "version": 1,
+            "acl": {
+                "owners": [
+                    "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                ],
+                "viewers": [
+                    "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                ]
+            },
+            "legal": {
+                "legaltags": [
+                    "opendes-demo-legaltag"
+                ],
+                "otherRelevantDataCountries": [
+                    "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                ],
+                "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+            },
+            "resourceHostRegionIDs": [
+                "srn:opendes:reference-data/OSDURegion:US-EAST:"
+            ],
+            "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+            "data": {
+                "FacilityTypeID": "srn:opendes:reference-data/FacilityType:Wellbore:",
+                "FacilityOperator": [
+                    {
+                        "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:"
+                    }
+                ],
+                "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                "SpatialLocation": [
+                    {
+                        "Coordinates": [
+                            {
+                                "x": -98.580887,
+                                "y": 35.6381829999999
+                            }
+                        ],
+                        "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Point:",
+                        "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:",
+                        "HorizontalCRSID": "srn:opendes:reference-data/HorizontalCRS:NAD27:",
+                        "HeightAboveGroundLevelUOMID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:"
+                    }
+                ],
+                "OperatingEnvironmentID": "srn:opendes:reference-data/OperatingEnvironment:onshore:",
+                "FacilityName": "IRETA 1-4-9XH",
+                "FacilityNameAlias": [
+                    {
+                        "AliasName": " IRETA 1-4-9XH",
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:Name:"
+                    },
+                    {
+                        "AliasName": "350112350400",
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:UWBI:"
+                    }
+                ],
+                "FacilityEvent": [
+                    {
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD:",
+                        "EffectiveDateTime": "2015-03-11T00:00:00-05:00"
+                    },
+                    {
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:",
+                        "EffectiveDateTime": "2015-05-18T00:00:00-06:00"
+                    }
+                ],
+                "WellID": "srn:opendes:master-data/Well:3501123504:",
+                "SequenceNumber": 1,
+                "VerticalMeasurements": [
+                    {
+                        "VerticalMeasurementID": "TD_1",
+                        "VerticalMeasurement": 0,
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Measured Depth:",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:",
+                        "VerticalReferenceID": "Drill Floor"
+                    },
+                    {
+                        "VerticalMeasurementID": "TD_2",
+                        "VerticalMeasurement": 0,
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:True Vertical Depth:",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:",
+                        "VerticalReferenceID": "Drill Floor"
+                    },
+                    {
+                        "VerticalMeasurementID": "Elev_1",
+                        "VerticalMeasurement": 1636,
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Drill Floor:",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:",
+                        "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:"
+                    },
+                    {
+                        "VerticalMeasurementID": "Elev_2",
+                        "VerticalMeasurement": 1606,
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Ground Level:",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:",
+                        "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:"
+                    }
+                ],
+                "TrajectoryTypeID": "srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:",
+                "DefaultVerticalMeasurementID": "",
+                "GeographicBottomHoleLocation": {
+                    "Coordinates": [
+                        {
+                            "x": -98.580887,
+                            "y": 35.6381829999999
+                        }
+                    ]
+                }
+            }
+        }
+
+    ],
+    "WorkflowID": "foo"
+}
diff --git a/tests/plugin-unit-tests/data/process_manifest_r3.py b/tests/plugin-unit-tests/data/process_manifest_r3.py
index c1d88c84111ee7a986b5b6796196bf7f5db4e77c..eaca093eb3d077c6aa8d0d1c25fafbe3abb01860 100644
--- a/tests/plugin-unit-tests/data/process_manifest_r3.py
+++ b/tests/plugin-unit-tests/data/process_manifest_r3.py
@@ -19,6 +19,220 @@ ACL_DICT = {'viewers': ['data.default.viewers@odes.osdu.test.net'],
 LEGAL_DICT = {'legaltags': ['odes-demo-legaltag'], 'otherRelevantDataCountries': ['FR', 'US', 'CA'],
               'status': 'compliant'}
 
+CONF_LOAD_WELL_13104 = {
+    "Payload": {
+        "authorization": "Bearer test",
+        "data-partition-id": "opendes",
+        "AppKey": "",
+        "kind_version": "3.0.0"
+    },
+    "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
+    "$filename": "load_Wellbore.1.0.0_350112350400.json",
+    "manifest": [
+        {
+
+            "id": "srn:opendes:master-data/Well:131041",
+            "version": 1,
+            "kind": "opendes:osdu:Well:0.3.0",
+            "groupType": "master-data",
+            "acl": {
+                "owners": [
+                    "ownergroup@testcompany.com"
+                ],
+                "viewers": [
+                    "viewgroup@testcompany.com"
+                ]
+            },
+            "legal": {
+                "legaltags": [
+                    "legaltag"
+                ],
+                "otherRelevantDataCountries": [
+                    "GB"
+                ]
+            },
+            "resourceObjectCreationDateTime": "2012-03-19T07:22Z",
+            "resourceVersionCreationDateTime": "2012-03-19T07:22Z",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:Public:1",
+            "data": {
+                "FacilityTypeID": "srn:opendes:reference-data/FacilityType:WELL:1",
+                "FacilityOperator": [
+                    {
+                        "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CHRYSAOR PRODUCTION (U.K.) LIMITED:1"
+                    }
+                ],
+                "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:UK_OGA:1",
+                "SpatialLocation": [
+                    {
+                        "Wgs84Coordinates": {
+                            "type": "FeatureCollection",
+                            "features": [
+                                {
+                                    "type": "Feature",
+                                    "geometry": {
+                                        "type": "Point",
+                                        "coordinates": [
+                                            1.896235806,
+                                            53.72433018
+                                        ]
+                                    }
+                                }
+                            ]
+                        }
+                    }
+                ],
+                "FacilityName": "48/10b-N2",
+                "FacilityNameAlias": [
+                    {
+                        "AliasName": "48/10b-N2",
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:WELL_NAME:1"
+                    }
+                ],
+                "FacilityState": [
+                    {
+                        "FacilityStateTypeID": "srn:opendes:reference-data/FacilityStateType:PLUGGED:1"
+                    }
+                ],
+                "FacilityEvent": [
+                    {
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD_DATE:1",
+                        "EffectiveDateTime": "2005-08-14T00:00:00"
+                    }
+                ],
+                "VerticalMeasurements": [
+                    {
+                        "VerticalMeasurementID": "RT",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:DEPTH_DATUM_ELEV:1",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:M:1"
+                    }
+                ]
+            }
+
+        }
+    ]
+}
+
+
+CONF_CONTRACTOR_TYPE = {
+    "Payload": {
+        "authorization": "Bearer test",
+        "data-partition-id": "opendes",
+        "AppKey": "",
+    },
+    "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
+    "manifest": [
+        {
+            "id": "srn:opendes:reference-data/ContractorType:1",
+            "kind": "opendes:osdu:ContractorType:1.0.0",
+            "groupType": "reference-data",
+            "version": 1,
+            "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"],
+                    "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]},
+            "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"],
+                      "status": "compliant"},
+            "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1",
+            "resourceHostRegionIDs": [
+                "srn:opendes:reference-data/OSDURegion:US-EAST:1"
+            ],
+            "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:1",
+            "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:1",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:1",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:1",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:1",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:1",
+            "data": {
+                "Name": "Recording",
+                "Description": "Performs data acquistion",
+                "Code": "Record"
+            }
+        },
+        {
+            "id": "srn:opendes:reference-data/ContractorType:1",
+            "kind": "opendes:osdu:ContractorType:1.0.0",
+            "groupType": "reference-data",
+            "version": 1,
+            "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"],
+                    "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]},
+            "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"],
+                      "status": "compliant"},
+            "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1",
+            "resourceHostRegionIDs": [
+                "srn:opendes:reference-data/OSDURegion:US-EAST:1"
+            ],
+            "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:1",
+            "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:1",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:1",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:1",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:1",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:1",
+            "data": {
+                "Name": "Line Clearing",
+                "Description": "Prepares onshore swath access",
+                "Code": "LineClear"
+            }
+        },
+        {
+            "id": "srn:opendes:reference-data/ContractorType:1",
+            "kind": "opendes:osdu:ContractorType:1.0.0",
+            "groupType": "reference-data",
+            "version": 1,
+            "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"],
+                    "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]},
+            "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"],
+                      "status": "compliant"},
+            "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1",
+            "resourceHostRegionIDs": [
+                "srn:opendes:reference-data/OSDURegion:US-EAST:1"
+            ],
+            "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:1",
+            "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:1",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:1",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:1",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:1",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:1",
+            "data": {
+                "Name": "Positioning",
+                "Description": "Establishes location of surface equipment",
+                "Code": "Position"
+            }
+        },
+        {
+            "id": "srn:opendes:reference-data/ContractorType:1",
+            "kind": "opendes:osdu:ContractorType:1.0.0",
+            "groupType": "reference-data",
+            "version": 1,
+            "acl": {"viewers": ["data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"],
+                    "owners": ["data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"]},
+            "legal": {"legaltags": ["opendes-demo-legaltag"], "otherRelevantDataCountries": ["US"],
+                      "status": "compliant"},
+            "resourceHomeRegionID": "srn:opendes:reference-data/OSDURegion:US-EAST:1",
+            "resourceHostRegionIDs": [
+                "srn:opendes:reference-data/OSDURegion:US-EAST:1"
+            ],
+            "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+            "resourceCurationStatus": "srn:opendes:reference-data/ResourceCurationStatus:CURATED:",
+            "resourceLifecycleStatus": "srn:opendes:reference-data/ResourceLifecycleStatus:LOADING:",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+            "data": {
+                "Name": "Data Processing",
+                "Description": "Transforms data",
+                "Code": "DataProc"
+            }
+        }
+    ]
+}
+
+
 CONF = {
     "Payload": {
         "authorization": "Bearer test",
@@ -29,17 +243,17 @@ CONF = {
     "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
     "$filename": "load_Wellbore.1.0.0_350112350400.json",
     "manifest": [
-{
-            "id": "srn:<namespace>:master-data/Wellbore:350112350400",
+        {
+            "id": "srn:opendes:master-data/Wellbore:350112350400",
             "kind": "opendes:osdu:Wellbore:0.3.0",
             "groupType": "master-data",
             "version": 1,
             "acl": {
                 "owners": [
-"data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                    "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
                 ],
                 "viewers": [
-"data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                    "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
                 ]
             },
             "legal": {
@@ -47,27 +261,27 @@ CONF = {
                     "opendes-demo-legaltag"
                 ],
                 "otherRelevantDataCountries": [
-                    "srn:<namespace>:master-data/GeoPoliticalEntity:USA:"
+                    "srn:opendes:master-data/GeoPoliticalEntity:USA:"
                 ],
-                "status": "srn:<namespace>:reference-data/LegalStatus:public:1111"
+                "status": "srn:opendes:reference-data/LegalStatus:public:1111"
             },
             "resourceHostRegionIDs": [
-                "srn:<namespace>:reference-data/OSDURegion:US-EAST:"
+                "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:<namespace>:reference-data/ResourceSecurityClassification:public:",
-            "source": "srn:<namespace>:master-data/Organisation:Oklahoma Corporation Commission:",
-            "existenceKind": "srn:<namespace>:reference-data/ExistenceKind:Active:",
-            "licenseState": "srn:<namespace>:reference-data/LicenseState:Unlicensed:",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
             "data": {
-                "FacilityTypeID": "srn:<namespace>:reference-data/FacilityType:Wellbore:",
+                "FacilityTypeID": "srn:opendes:reference-data/FacilityType:Wellbore:",
                 "FacilityOperator": [
                     {
-                        "FacilityOperatorOrganisationID": "srn:<namespace>:master-data/Organisation:CONTINENTAL RESOURCES INC:"
+                        "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:"
                     }
                 ],
-                "DataSourceOrganisationID": "srn:<namespace>:master-data/Organisation:Oklahoma Corporation Commission:",
+                "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
                 "SpatialLocation": [
                     {
                         "Coordinates": [
@@ -76,71 +290,71 @@ CONF = {
                                 "y": 35.6381829999999
                             }
                         ],
-                        "SpatialGeometryTypeID": "srn:<namespace>:reference-data/SpatialGeometryType:Point:",
-                        "VerticalCRSID": "srn:<namespace>:reference-data/VerticalCRS:MSL:",
-                        "HorizontalCRSID": "srn:<namespace>:reference-data/HorizontalCRS:NAD27:",
-                        "HeightAboveGroundLevelUOMID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:"
+                        "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:<namespace>:reference-data/OperatingEnvironment:onshore:",
+                "OperatingEnvironmentID": "srn:opendes:reference-data/OperatingEnvironment:onshore:",
                 "FacilityName": "IRETA 1-4-9XH",
                 "FacilityNameAlias": [
                     {
                         "AliasName": " IRETA 1-4-9XH",
-                        "AliasNameTypeID": "srn:<namespace>:reference-data/AliasNameType:Name:"
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:Name:"
                     },
                     {
                         "AliasName": "350112350400",
-                        "AliasNameTypeID": "srn:<namespace>:reference-data/AliasNameType:UWBI:"
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:UWBI:"
                     }
                 ],
                 "FacilityEvent": [
                     {
-                        "FacilityEventTypeID": "srn:<namespace>:reference-data/FacilityEventType:SPUD:",
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD:",
                         "EffectiveDateTime": "2015-03-11T00:00:00-05:00"
                     },
                     {
-                        "FacilityEventTypeID": "srn:<namespace>:reference-data/FacilityEventType:DRILLING FINISH:",
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:",
                         "EffectiveDateTime": "2015-05-18T00:00:00-06:00"
                     }
                 ],
-                "WellID": "srn:<namespace>:master-data/Well:3501123504:",
+                "WellID": "srn:opendes:master-data/Well:3501123504:",
                 "SequenceNumber": 1,
                 "VerticalMeasurements": [
                     {
                         "VerticalMeasurementID": "TD_1",
                         "VerticalMeasurement": 0,
-                        "VerticalMeasurementTypeID": "srn:<namespace>:reference-data/VerticalMeasurementType:Total Depth:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:Measured Depth:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
+                        "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:<namespace>:reference-data/VerticalMeasurementType:Total Depth:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:True Vertical Depth:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
+                        "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:<namespace>:reference-data/VerticalMeasurementType:Drill Floor:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:Elevation:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
-                        "VerticalCRSID": "srn:<namespace>:reference-data/VerticalCRS:MSL:"
+                        "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:<namespace>:reference-data/VerticalMeasurementType:Ground Level:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:Elevation:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
-                        "VerticalCRSID": "srn:<namespace>:reference-data/VerticalCRS:MSL:"
+                        "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:<namespace>:reference-data/WellboreTrajectoryType:Horizontal:",
+                "TrajectoryTypeID": "srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:",
                 "DefaultVerticalMeasurementID": "",
                 "GeographicBottomHoleLocation": {
                     "Coordinates": [
@@ -157,6 +371,61 @@ CONF = {
     "WorkflowID": "foo"
 }
 
+CONF_TEST_REFERENCE = {
+    "Payload": {
+        "authorization": "Bearer test",
+        "data-partition-id": "opendes",
+        "AppKey": "",
+        "kind_version": "3.0.0"
+    },
+    "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
+    "$filename": "load_Wellbore.1.0.0_350112350400.json",
+    "manifest": [
+
+        {
+            "kind": "opendes:osdu:TestReference:1.0.1",
+            "groupType": "reference-data",
+            "acl": {
+                "owners": [
+                    "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                ],
+                "viewers": [
+                    "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                ]
+            },
+            "legal": {
+                "legaltags": [
+                    "opendes-demo-legaltag"
+                ],
+                "otherRelevantDataCountries": [
+                    "US"
+                ],
+                "status": "compliant"
+            },
+            "data": {
+                "Name": "TestReference",
+                "Description": "A meaningful description of this TestReference.",
+                "Schema": "http://json-schema.org/draft-07/schema#",
+                "SchemaID": "https://schema.osdu.opengroup.org/json/reference-data/TestReference.1.0.0.json",
+                "SchemaKind": "osdu:osdu:TestReference:1.0.0",
+                "GroupType": "reference-data",
+                "IsReferenceValueType": True,
+                "GovernanceAuthorities": [
+                    "$$srn:NAMESPACE$$:reference-data/OrganisationType:osdu"
+                ],
+                "NaturalKeys": [
+                    "data.Code",
+                    "data.Name"
+                ],
+                "GovernanceModel": "LOCAL"
+            }
+        }
+
+    ],
+    "WorkflowID": "foo"
+}
+
+
 CONF2 = {
     "WorkflowID": "{{workflow_id}}",
     "Payload": {
@@ -165,7 +434,7 @@ CONF2 = {
     },
     "manifest": [
         {
-            "id": "srn:<namespace>:master-data/Wellbore:350112350400",
+            "id": "srn:opendes:master-data/Wellbore:350112350400",
             "kind": "osdu:osdu:Wellbore:0.3.0",
             "groupType": "master-data",
             "version": 1,
@@ -182,27 +451,27 @@ CONF2 = {
                     "legaltag1"
                 ],
                 "OtherRelevantDataCountries": [
-                    "srn:<namespace>:master-data/GeoPoliticalEntity:USA:"
+                    "srn:opendes:master-data/GeoPoliticalEntity:USA:"
                 ],
-                "Status": "srn:<namespace>:reference-data/LegalStatus:public:"
+                "Status": "srn:opendes:reference-data/LegalStatus:public:"
             },
             "resourceHostRegionIDs": [
-                "srn:<namespace>:reference-data/OSDURegion:US-EAST:"
+                "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:<namespace>:reference-data/ResourceSecurityClassification:public:",
-            "source": "srn:<namespace>:master-data/Organisation:Oklahoma Corporation Commission:",
-            "existenceKind": "srn:<namespace>:reference-data/ExistenceKind:Active:",
-            "licenseState": "srn:<namespace>:reference-data/LicenseState:Unlicensed:",
+            "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+            "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+            "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+            "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
             "data": {
-                "FacilityTypeID": "srn:<namespace>:reference-data/FacilityType:Wellbore:",
+                "FacilityTypeID": "srn:opendes:reference-data/FacilityType:Wellbore:",
                 "FacilityOperator": [
                     {
-                        "FacilityOperatorOrganisationID": "srn:<namespace>:master-data/Organisation:CONTINENTAL RESOURCES INC:"
+                        "FacilityOperatorOrganisationID": "srn:opendes:master-data/Organisation:CONTINENTAL RESOURCES INC:"
                     }
                 ],
-                "DataSourceOrganisationID": "srn:<namespace>:master-data/Organisation:Oklahoma Corporation Commission:",
+                "DataSourceOrganisationID": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
                 "SpatialLocation": [
                     {
                         "Coordinates": [
@@ -211,71 +480,71 @@ CONF2 = {
                                 "y": 35.6381829999999
                             }
                         ],
-                        "SpatialGeometryTypeID": "srn:<namespace>:reference-data/SpatialGeometryType:Point:",
-                        "VerticalCRSID": "srn:<namespace>:reference-data/VerticalCRS:MSL:",
-                        "HorizontalCRSID": "srn:<namespace>:reference-data/HorizontalCRS:NAD27:",
-                        "HeightAboveGroundLevelUOMID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:"
+                        "SpatialGeometryTypeID": "srn:opendes:reference-data/SpatialGeometryType:Point:1",
+                        "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:1",
+                        "HorizontalCRSID": "srn:opendes:reference-data/HorizontalCRS:NAD27:1",
+                        "HeightAboveGroundLevelUOMID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1"
                     }
                 ],
-                "OperatingEnvironmentID": "srn:<namespace>:reference-data/OperatingEnvironment:onshore:",
+                "OperatingEnvironmentID": "srn:opendes:reference-data/OperatingEnvironment:onshore:1",
                 "FacilityName": "IRETA 1-4-9XH",
                 "FacilityNameAlias": [
                     {
                         "AliasName": " IRETA 1-4-9XH",
-                        "AliasNameTypeID": "srn:<namespace>:reference-data/AliasNameType:Name:"
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:Name:1"
                     },
                     {
                         "AliasName": "350112350400",
-                        "AliasNameTypeID": "srn:<namespace>:reference-data/AliasNameType:UWBI:"
+                        "AliasNameTypeID": "srn:opendes:reference-data/AliasNameType:UWBI:1"
                     }
                 ],
                 "FacilityEvent": [
                     {
-                        "FacilityEventTypeID": "srn:<namespace>:reference-data/FacilityEventType:SPUD:",
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:SPUD:1",
                         "EffectiveDateTime": "2015-03-11T00:00:00-05:00"
                     },
                     {
-                        "FacilityEventTypeID": "srn:<namespace>:reference-data/FacilityEventType:DRILLING FINISH:",
+                        "FacilityEventTypeID": "srn:opendes:reference-data/FacilityEventType:DRILLING FINISH:1",
                         "EffectiveDateTime": "2015-05-18T00:00:00-06:00"
                     }
                 ],
-                "WellID": "srn:<namespace>:master-data/Well:3501123504:",
+                "WellID": "srn:opendes:master-data/Well:3501123504:1",
                 "SequenceNumber": 1,
                 "VerticalMeasurements": [
                     {
                         "VerticalMeasurementID": "TD_1",
                         "VerticalMeasurement": 0,
-                        "VerticalMeasurementTypeID": "srn:<namespace>:reference-data/VerticalMeasurementType:Total Depth:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:Measured Depth:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:1",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Measured Depth:1",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1",
                         "VerticalReferenceID": "Drill Floor"
                     },
                     {
                         "VerticalMeasurementID": "TD_2",
                         "VerticalMeasurement": 0,
-                        "VerticalMeasurementTypeID": "srn:<namespace>:reference-data/VerticalMeasurementType:Total Depth:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:True Vertical Depth:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Total Depth:1",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:True Vertical Depth:1",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1",
                         "VerticalReferenceID": "Drill Floor"
                     },
                     {
                         "VerticalMeasurementID": "Elev_1",
                         "VerticalMeasurement": 1636,
-                        "VerticalMeasurementTypeID": "srn:<namespace>:reference-data/VerticalMeasurementType:Drill Floor:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:Elevation:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
-                        "VerticalCRSID": "srn:<namespace>:reference-data/VerticalCRS:MSL:"
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Drill Floor:1",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:1",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1",
+                        "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:1"
                     },
                     {
                         "VerticalMeasurementID": "Elev_2",
                         "VerticalMeasurement": 1606,
-                        "VerticalMeasurementTypeID": "srn:<namespace>:reference-data/VerticalMeasurementType:Ground Level:",
-                        "VerticalMeasurementPathID": "srn:<namespace>:reference-data/VerticalMeasurementPath:Elevation:",
-                        "VerticalMeasurementUnitOfMeasureID": "srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:",
-                        "VerticalCRSID": "srn:<namespace>:reference-data/VerticalCRS:MSL:"
+                        "VerticalMeasurementTypeID": "srn:opendes:reference-data/VerticalMeasurementType:Ground Level:1",
+                        "VerticalMeasurementPathID": "srn:opendes:reference-data/VerticalMeasurementPath:Elevation:1",
+                        "VerticalMeasurementUnitOfMeasureID": "srn:opendes:reference-data/UnitOfMeasure:ft[US]:1",
+                        "VerticalCRSID": "srn:opendes:reference-data/VerticalCRS:MSL:1"
                     }
                 ],
-                "TrajectoryTypeID": "srn:<namespace>:reference-data/WellboreTrajectoryType:Horizontal:",
+                "TrajectoryTypeID": "srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:1",
                 "DefaultVerticalMeasurementID": "",
                 "GeographicBottomHoleLocation": {
                     "Coordinates": [
@@ -302,11 +571,11 @@ TEST_SCHEMA = {
             "description": "The SRN which identifies this OSDU resource object without version.",
             "title": "Entity ID",
             "type": "string",
-            "pattern": "^srn:<namespace>:master-data\\/Wellbore:[^:]+$",
-            "example": "srn:<namespace>:master-data/Wellbore:2adac27b-5d84-5bcd-89f2-93ee709c06d9"
+            "pattern": "^srn:opendes:master-data\\/Wellbore:[^:]+$",
+            "example": "srn:opendes:master-data/Wellbore:2adac27b-5d84-5bcd-89f2-93ee709c06d9"
         },
         "kind": {
-            "description": "The schema identification for the OSDU resource object following the pattern <Namespace>:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.",
+            "description": "The schema identification for the OSDU resource object following the pattern opendes:<Source>:<Type>:<VersionMajor>.<VersionMinor>.<VersionPatch>. The versioning scheme follows the semantic versioning, https://semver.org/.",
             "title": "Entity Kind",
             "type": "string",
             "pattern": "^[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[A-Za-z0-9-_]+:[0-9]+.[0-9]+.[0-9]+$",
@@ -338,7 +607,7 @@ TEST_SCHEMA = {
             "description": "The name of the home [cloud environment] region for this OSDU resource object.",
             "title": "Resource Home Region ID",
             "type": "string",
-            "pattern": "^srn:<namespace>:reference-data\\/OSDURegion:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:reference-data\\/OSDURegion:[^:]+:[0-9]*$"
         },
         "resourceHostRegionIDs": {
             "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.",
@@ -346,7 +615,7 @@ TEST_SCHEMA = {
             "type": "array",
             "items": {
                 "type": "string",
-                "pattern": "^srn:<namespace>:reference-data\\/OSDURegion:[^:]+:[0-9]*$"
+                "pattern": "^srn:opendes:reference-data\\/OSDURegion:[^:]+:[0-9]*$"
             }
         },
         "resourceObjectCreationDateTime": {
@@ -365,19 +634,19 @@ TEST_SCHEMA = {
             "description": "Describes the current Curation status.",
             "title": "Resource Curation Status",
             "type": "string",
-            "pattern": "^srn:<namespace>:reference-data\\/ResourceCurationStatus:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:reference-data\\/ResourceCurationStatus:[^:]+:[0-9]*$"
         },
         "resourceLifecycleStatus": {
             "description": "Describes the current Resource Lifecycle status.",
             "title": "Resource Lifecycle Status",
             "type": "string",
-            "pattern": "^srn:<namespace>:reference-data\\/ResourceLifecycleStatus:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:reference-data\\/ResourceLifecycleStatus:[^:]+:[0-9]*$"
         },
         "resourceSecurityClassification": {
             "description": "Classifies the security level of the resource.",
             "title": "Resource Security Classification",
             "type": "string",
-            "pattern": "^srn:<namespace>:reference-data\\/ResourceSecurityClassification:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:reference-data\\/ResourceSecurityClassification:[^:]+:[0-9]*$"
         },
         "ancestry": {
             "description": "The links to data, which constitute the inputs.",
@@ -388,19 +657,19 @@ TEST_SCHEMA = {
             "description": "Where did the data resource originate? This could be many kinds of entities, such as company, agency, team or individual.",
             "title": "Data Source",
             "type": "string",
-            "pattern": "^srn:<namespace>:master-data\\/Organisation:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:master-data\\/Organisation:[^:]+:[0-9]*$"
         },
         "existenceKind": {
             "description": "Where does this data resource sit in the cradle-to-grave span of its existence?",
             "title": "Existence Kind",
             "type": "string",
-            "pattern": "^srn:<namespace>:reference-data\\/ExistenceKind:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:reference-data\\/ExistenceKind:[^:]+:[0-9]*$"
         },
         "licenseState": {
             "description": "Indicates what kind of ownership Company has over data.",
             "title": "License State",
             "type": "string",
-            "pattern": "^srn:<namespace>:reference-data\\/LicenseState:[^:]+:[0-9]*$"
+            "pattern": "^srn:opendes:reference-data\\/LicenseState:[^:]+:[0-9]*$"
         },
         "data": {
             "allOf": [
@@ -438,68 +707,68 @@ EMPTY_MANIFEST = {
 }
 
 EXPECTED_RECORD = [{'legal': {'LegalTags': ['legaltag1'], 'OtherRelevantDataCountries': [
-    'srn:<namespace>:master-data/GeoPoliticalEntity:USA:'],
-                              'Status': 'srn:<namespace>:reference-data/LegalStatus:public:'},
+    'srn:opendes:master-data/GeoPoliticalEntity:USA:'],
+                              'Status': 'srn:opendes:reference-data/LegalStatus:public:'},
                     'acl': {'Owners': ['users@odes.osdu.joonix.net'],
                             'Viewers': ['users@odes.osdu.joonix.net']},
                     'kind': 'osdu:osdu:Wellbore:0.3.0', 'id': '',
-                    'data': {'id': 'srn:<namespace>:master-data/Wellbore:350112350400',
+                    'data': {'id': 'srn:opendes:master-data/Wellbore:350112350400',
                              'groupType': 'master-data', 'version': 1, 'resourceHostRegionIDs': [
-                            'srn:<namespace>:reference-data/OSDURegion:US-EAST:'],
+                            '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:<namespace>:reference-data/ResourceSecurityClassification:public:',
-                             'source': 'srn:<namespace>:master-data/Organisation:Oklahoma Corporation Commission:',
-                             'existenceKind': 'srn:<namespace>:reference-data/ExistenceKind:Active:',
-                             'licenseState': 'srn:<namespace>:reference-data/LicenseState:Unlicensed:',
+                             'resourceSecurityClassification': 'srn:opendes:reference-data/ResourceSecurityClassification:public:',
+                             'source': 'srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:',
+                             'existenceKind': 'srn:opendes:reference-data/ExistenceKind:Active:',
+                             'licenseState': 'srn:opendes:reference-data/LicenseState:Unlicensed:',
                              'data': {
-                                 'FacilityTypeID': 'srn:<namespace>:reference-data/FacilityType:Wellbore:',
+                                 'FacilityTypeID': 'srn:opendes:reference-data/FacilityType:Wellbore:',
                                  'FacilityOperator': [{
-                                                          'FacilityOperatorOrganisationID': 'srn:<namespace>:master-data/Organisation:CONTINENTAL RESOURCES INC:'}],
-                                 'DataSourceOrganisationID': 'srn:<namespace>:master-data/Organisation:Oklahoma Corporation Commission:',
+                                     '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:<namespace>:reference-data/SpatialGeometryType:Point:',
-                                      'VerticalCRSID': 'srn:<namespace>:reference-data/VerticalCRS:MSL:',
-                                      'HorizontalCRSID': 'srn:<namespace>:reference-data/HorizontalCRS:NAD27:',
-                                      'HeightAboveGroundLevelUOMID': 'srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:'}],
-                                 'OperatingEnvironmentID': 'srn:<namespace>:reference-data/OperatingEnvironment:onshore:',
+                                      '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:<namespace>:reference-data/AliasNameType:Name:'},
+                                      'AliasNameTypeID': 'srn:opendes:reference-data/AliasNameType:Name:'},
                                      {'AliasName': '350112350400',
-                                      'AliasNameTypeID': 'srn:<namespace>:reference-data/AliasNameType:UWBI:'}],
+                                      'AliasNameTypeID': 'srn:opendes:reference-data/AliasNameType:UWBI:'}],
                                  'FacilityEvent': [{
-                                                       'FacilityEventTypeID': 'srn:<namespace>:reference-data/FacilityEventType:SPUD:',
-                                                       'EffectiveDateTime': '2015-03-11T00:00:00-05:00'},
-                                                   {
-                                                       'FacilityEventTypeID': 'srn:<namespace>:reference-data/FacilityEventType:DRILLING FINISH:',
-                                                       'EffectiveDateTime': '2015-05-18T00:00:00-06:00'}],
-                                 'WellID': 'srn:<namespace>:master-data/Well:3501123504:',
+                                     '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:<namespace>:reference-data/VerticalMeasurementType:Total Depth:',
-                                      'VerticalMeasurementPathID': 'srn:<namespace>:reference-data/VerticalMeasurementPath:Measured Depth:',
-                                      'VerticalMeasurementUnitOfMeasureID': 'srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:',
+                                      '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:<namespace>:reference-data/VerticalMeasurementType:Total Depth:',
-                                      'VerticalMeasurementPathID': 'srn:<namespace>:reference-data/VerticalMeasurementPath:True Vertical Depth:',
-                                      'VerticalMeasurementUnitOfMeasureID': 'srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:',
+                                      '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:<namespace>:reference-data/VerticalMeasurementType:Drill Floor:',
-                                      'VerticalMeasurementPathID': 'srn:<namespace>:reference-data/VerticalMeasurementPath:Elevation:',
-                                      'VerticalMeasurementUnitOfMeasureID': 'srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:',
-                                      'VerticalCRSID': 'srn:<namespace>:reference-data/VerticalCRS:MSL:'},
+                                      '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:<namespace>:reference-data/VerticalMeasurementType:Ground Level:',
-                                      'VerticalMeasurementPathID': 'srn:<namespace>:reference-data/VerticalMeasurementPath:Elevation:',
-                                      'VerticalMeasurementUnitOfMeasureID': 'srn:<namespace>:reference-data/UnitOfMeasure:ft[US]:',
-                                      'VerticalCRSID': 'srn:<namespace>:reference-data/VerticalCRS:MSL:'}],
-                                 'TrajectoryTypeID': 'srn:<namespace>:reference-data/WellboreTrajectoryType:Horizontal:',
+                                      'VerticalMeasurementTypeID': 'srn:opendes:reference-data/VerticalMeasurementType:Ground Level:',
+                                      'VerticalMeasurementPathID': 'srn:opendes:reference-data/VerticalMeasurementPath:Elevation:',
+                                      'VerticalMeasurementUnitOfMeasureID': 'srn:opendes:reference-data/UnitOfMeasure:ft[US]:',
+                                      'VerticalCRSID': 'srn:opendes:reference-data/VerticalCRS:MSL:'}],
+                                 'TrajectoryTypeID': 'srn:opendes:reference-data/WellboreTrajectoryType:Horizontal:',
                                  'DefaultVerticalMeasurementID': '',
                                  'GeographicBottomHoleLocation': {
                                      'Coordinates': [{'x': -98.580887, 'y': 35.6381829999999}]}}}},
diff --git a/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json b/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json
new file mode 100644
index 0000000000000000000000000000000000000000..69463987d472af381678e8e3e15460108c36b688
--- /dev/null
+++ b/tests/plugin-unit-tests/data/workProduct/SeismicTraceData.json
@@ -0,0 +1,423 @@
+{
+    "Payload": {
+        "authorization": "Bearer test",
+        "data-partition-id": "opendes",
+        "AppKey": "",
+        "kind_version": "3.0.0"
+    },
+    "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
+    "$filename": "load_Wellbore.1.0.0_350112350400.json",
+    "manifest": [
+        {
+            "WorkProduct": {
+                "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"
+                ]
+            },
+            "WorkProductComponents": [
+                {
+                    "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"
+                    ]
+                }
+            ],
+            "Files": [
+                {
+                    "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": {
+                        "SchemaFormatTypeID": "srn:opendes:reference-data/SchemaFormatType:SEG-Y Seismic Trace Data:",
+                        "PreLoadFilePath": "C:\\Seismic\\ST0202R08_PS_PSDM_RAW_PP_TIME.MIG_RAW.POST_STACK.3D.JS-017534.segy",
+                        "FileSource": "",
+                        "FileSize": 277427976,
+                        "EncodingFormatTypeID": "srn:opendes:reference-data/EncodingFormatType:segy:",
+                        "Endian": "BIG",
+                        "Checksum": "c9df7234d5d0a7a2c2676ee2e2aa48b6",
+                        "VectorHeaderMapping": [
+                            {
+                                "KeyName": "srn:opendes:reference-data/HeaderKeyName:INLINE:",
+                                "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:",
+                                "WordWidth": 4,
+                                "Position": 189
+                            },
+                            {
+                                "KeyName": "srn:opendes:reference-data/HeaderKeyName:CROSSLINE:",
+                                "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:",
+                                "WordWidth": 4,
+                                "Position": 193
+                            },
+                            {
+                                "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPX:",
+                                "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:",
+                                "WordWidth": 4,
+                                "Position": 181,
+                                "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:",
+                                "ScalarIndicator": "OVERRIDE",
+                                "ScalarOverride": 100.0
+                            },
+                            {
+                                "KeyName": "srn:opendes:reference-data/HeaderKeyName:CMPY:",
+                                "WordFormat": "srn:opendes:reference-data/WordFormatType:INT:",
+                                "WordWidth": 4,
+                                "Position": 185,
+                                "UoM": "srn:opendes:reference-data/UnitOfMeasure:M:",
+                                "ScalarIndicator": "OVERRIDE",
+                                "ScalarOverride": 100.0
+                            }
+                        ]
+                    },
+                    "AssociativeID": "f-1"
+                }
+            ]
+        }
+    ],
+    "WorkflowID": "foo"
+}
diff --git a/tests/plugin-unit-tests/test_process_manifest_r2_op.py b/tests/plugin-unit-tests/test_process_manifest_r2.py
similarity index 89%
rename from tests/plugin-unit-tests/test_process_manifest_r2_op.py
rename to tests/plugin-unit-tests/test_process_manifest_r2.py
index e0932e1e6d2d47b38fd8ead49bd31bbcb2fd6fd3..e794084fdc45f847da84b56399fed23e07beea5b 100644
--- a/tests/plugin-unit-tests/test_process_manifest_r2_op.py
+++ b/tests/plugin-unit-tests/test_process_manifest_r2.py
@@ -23,7 +23,7 @@ import pytest
 sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins")
 
 from data import process_manifest_r2_op as test_data
-from operators import process_manifest_r2_op
+from operators import process_manifest_r2
 
 
 @pytest.mark.parametrize(
@@ -34,7 +34,7 @@ from operators import process_manifest_r2_op
     ]
 )
 def test_determine_data_type(test_input, expected):
-   data_type = process_manifest_r2_op.determine_data_type(test_input)
+   data_type = process_manifest_r2.determine_data_type(test_input)
    assert data_type == expected
 
 
@@ -50,7 +50,7 @@ def test_determine_data_type(test_input, expected):
 def test_process_file_items(data_type, loaded_conf, conf_payload, expected_file_result):
    file_id_regex = re.compile(r"srn\:file/" + data_type + r"\:\d+\:")
    expected_file_list = expected_file_result[0]
-   file_list, file_ids = process_manifest_r2_op.process_file_items(loaded_conf, conf_payload)
+   file_list, file_ids = process_manifest_r2.process_file_items(loaded_conf, conf_payload)
    for i in file_ids:
       assert file_id_regex.match(i)
 
diff --git a/tests/plugin-unit-tests/test_process_manifest_r3.py b/tests/plugin-unit-tests/test_process_manifest_r3.py
index e7a8b8123ee8cb55d37f7bb87b540bfbe99b88f8..8129f55048c598490520aa9e48f4ee9f99020431 100644
--- a/tests/plugin-unit-tests/test_process_manifest_r3.py
+++ b/tests/plugin-unit-tests/test_process_manifest_r3.py
@@ -15,6 +15,7 @@
 
 
 import copy
+import json
 import os
 import sys
 
@@ -30,35 +31,39 @@ from operators import process_manifest_r3
 @pytest.mark.parametrize(
     "input",
     [
-        pytest.param(test_data.CONF, id="Valid manifest"),
-        # pytest.param(test_data.EMPTY_MANIFEST, marks=pytest.mark.xfail, id="Empty manifest")
+        # pytest.param(test_data.CONF, id="Valid manifest"),
+        # pytest.param(test_data.EMPTY_MANIFEST, marks=pytest.mark.xfail, id="Empty manifest"),
+        pytest.param("data/workProduct/SeismicTraceData.json", id="Valid WorkProduct"),
+        pytest.param("data/master/Wellbore.0.3.0.json", id="Valid master")
     ]
 )
 def test_create_manifest_records(input):
-    conf = copy.deepcopy(input)
+    with open(input) as f:
+        conf = json.load(f)
     context = process_manifest_r3.Context.populate(conf)
     manifest_processor = process_manifest_r3.ManifestProcessor(
-        storage_url="https://storage-jvmvia5dea-uc.a.run.app/api/storage/v2/records",
-        dagrun_conf=test_data.CONF,
+        storage_url="",
+        dagrun_conf=conf,
         context=context
     )
-    manifest_processor.create_manifest_records()
+    records = manifest_processor.create_manifest_records()
+    records = json.dumps(records, indent=4)
+    print(records)
 
 
 @pytest.mark.parametrize(
     "schema,manifest",
     [
-        pytest.param(test_data.TEST_SCHEMA, test_data.CONF, id="Valid manifest"),
+        pytest.param(test_data.TEST_SCHEMA, test_data.CONF, marks=pytest.mark.xfail,
+                     id="Valid manifest"),
     ]
 )
 def test_schema_validator(schema, manifest):
     conf = copy.deepcopy(manifest)
-    context = process_manifest_r3.Context.populate(manifest)
-    SCHEMA_SERVICE = "https://os-schema-jvmvia5dea-uc.a.run.app/api/schema-service/v1/schema"
-    context.data_partition_id = "opendes"
+    context = process_manifest_r3.Context.populate(conf)
     validator = process_manifest_r3.SchemaValidator(
-        SCHEMA_SERVICE,
+        "",
         conf,
         context
     )
-    validator._validate_schema(schema, manifest)
+    validator._validate_schema(manifest["manifest"][0], schema)