diff --git a/.gitignore b/.gitignore
index a76ac17e22bbcc2148c486865aab202ae79201f0..d4eaab722280ec3d8a06f510ffd5d16d5ef2309c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -30,6 +30,7 @@
 **/.installed.cfg
 **/*.egg
 **/MANIFEST
+**/venv
 
 # ignore coverage.py
 htmlcov/*
diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml
index d7b4e0af75e8e2055f0d13919bcb5c69777fabca..cf6b94ecde861b007c0ddffc934b9c7709194cbc 100644
--- a/.gitlab-ci.yml
+++ b/.gitlab-ci.yml
@@ -22,7 +22,7 @@ stages:
   - deploy
 
 pylint:
-  image: johnybear/osdu-airflow:python36-2
+  image: johnybear/osdu-airflow:airflow.1.10.14
   stage: linters
   allow_failure: true
   script:
@@ -32,7 +32,7 @@ pylint:
     - exit ${EXIT_CODE}
 
 isort:
-  image: johnybear/osdu-airflow:python36-2
+  image: johnybear/osdu-airflow:airflow.1.10.14
   allow_failure: true
   stage: linters
   script:
@@ -42,7 +42,7 @@ isort:
 
 test_dags:
   stage: test_dags
-  image: johnybear/osdu-airflow:python36-2
+  image: johnybear/osdu-airflow:airflow.1.10.14
   script:
     - chmod +x tests/test_dags.sh
     - tests/./test_dags.sh  || EXIT_CODE=$?
@@ -55,12 +55,13 @@ test_dags:
 
 unit_tests:
   stage: unit_tests
-  image: johnybear/osdu-airflow:python36-2
+  image: johnybear/osdu-airflow:airflow.1.10.14
   script:
     - chmod +x tests/unit_tests.sh
     - tests/./unit_tests.sh  || EXIT_CODE=$?
     - exit ${EXIT_CODE}
 
+# TODO: Think about how rsync must look. At the moment it looks messy.
 osdu-gcp-deploy:
   stage: deploy
   script:
@@ -70,6 +71,7 @@ osdu-gcp-deploy:
     - gsutil -m rsync -d -R dags/libs  $OSDU_GCP_DEPL_TARGET/dags/libs
     - gsutil -m rsync -x "dags/providers*" -d -R dags  $OSDU_GCP_DEPL_TARGET/dags/ingestion
     - gsutil -m rsync -d -R dags/providers  $OSDU_GCP_DEPL_TARGET/dags/providers
+    - gsutil cp dags/.airflowignore $OSDU_GCP_DEPL_TARGET/dags/
     - gsutil -m rsync -R plugins $OSDU_GCP_DEPL_TARGET/plugins
   except:
     - /^release\/*/
diff --git a/README.md b/README.md
index 12c4e2afcb79262f432f52082f1c03521d449b2e..6eec16153a37b50a8b2c4d8c8dfca2793320d516 100644
--- a/README.md
+++ b/README.md
@@ -29,17 +29,11 @@
 
 ## Introduction
 
-The OSDU R2 Prototype includes a Workflow Engine, an implementation of Apache Airflow, to orchestrate business
-processes. In particular, the Workflow Engine handles ingestion of opaque and well log .las files in OSDU R2.
-
-The Workflow Engine encompasses the following components:
-
-* Opaque Ingestion DAG
-* OSDU Ingestion DAG
-* Workflow Status Operator
-* Stale Jobs Scheduler
-* Workflow Finished Sensor Operator
+The project is a set of Apache Airflow DAGs implementations to orchestrate data ingestion within OSDU platform.
+The following DAGs are implemented:
 
+* Osdu_ingest - R3 Manifest Ingestion DAG
+* Osdu_ingest_r2 - R2 Manifest Ingestion DAG
 
 ## Deployment
 
@@ -48,15 +42,14 @@ GCP provides Cloud Composer a fully managed workflow orchestration service built
 
 To deploy the Ingestion DAGs on GCP Cloud Composer just upload files from */src* folder into *DAGS_FOLDER* and *PLUGINS_FOLDER* accordingly into the DAG bucket that provided by Composer environment. [More info in documentation.](https://cloud.google.com/composer/docs/quickstart#uploading_the_dag_to)
 
-*DAGS_FOLDER* and *FLUGINS_FOLDER* are setting up in airflow.cfg file.
+*DAGS_FOLDER* and *PLUGINS_FOLDER* are setting up by Composer itself.
 
 According to the [DAG implementation details](#dag-implementation-details) need to put [osdu_api] directory into the *DAGS_FOLDER*. Moreover, all required variables have to be set in Airflow meta store by Variables mechanism. [List of the required variables](#required-variables).
 
-### Installing Python Dependencies
+#### Installing Python Dependencies
 Environment dependencies might be installed by several ways:
-1. Installing a Python dependency from PyPI. Cloud Composer picks up *requirements.txt* file from the DAGs bucket.
-2. Setting up an environment into the Cloud Composer Console.
-3. Installing local Python library. Put your dependencies into *DAG_FOLDER/libs* directory. Airflow automatically adds *DAG_FOLDER* and *PLUGINS_FOLDER* to the *PATH*.
+1. Setting up an environment into the Cloud Composer Console.
+2. Installing local Python library. Put your dependencies into *DAG_FOLDER/libs* directory. Airflow automatically adds *DAG_FOLDER* and *PLUGINS_FOLDER* to the *PATH*.
 
 
 ## DAG Implementation Details
@@ -67,24 +60,34 @@ cloud platforms. This Python SDK is located in a separate [os-python-sdk] folder
 
 
 ## Required Variables
+
+### Common naming convention
+Some variables are defined using Airflow Variables.
+Variable should has prefix which define where variable are used:
+- **core__** - use in common part of DAGs;
+- **gcp__**, **azure__**, **ibm__**, **aws__** - use in cloud-specific modules of DAGs;
+- **sdk__** - pass to Python SDK.
+
+If variable defines URL to internal services it should have suffix which show the completeness of the URL:
+- **__url** - the variable should contain full path to service API endpoint;
+- **__host** - the variable should contain just service host value. The full path to service API endpoint constructed inside operators.
+
+
 ### Internal Services
-Some of the operators send requests to internal services. Hosts and endpoints are sepcified into Airflow Variables.
 |Variable |Value Description|
 |---|---|
-| storage_url  | Storage Service URL |
-| search_url |  Search Service URL |
-| update_status_ep | Endpoint to Workflow Service hook call|
+| core__service__storage__url  | Storage Service API endpoint to save records |
+| core__service__search__url |  Search Service API endpoint to search queries |
+| core__service__workflow__host | Workflow Service host |
+| core__service__workflow__url | (Deprecated) Workflow Service API endpoint to update status |
+| core__service__file__host | File Service host |
+| core__service__schema__url | Schema Service API endpoint to get schema by Id |
 
-### Configuration
-|Variable |Value Description|
-|---|---|
-| dataload_config_path| Path to dataload.ini file|
 
-### OSDU Python SDK
+### Configuration
 |Variable |Value Description|
 |---|---|
-| provider  |  Need to properly initialize OSDU SDK  |
-|entitlements_module_name | Need to properly  initialize OSDU SDK |
+| core__config__dataload_config_path| Path to dataload.ini file. Used in R2 manifest ingestion|
 
 
 ## Testing
diff --git a/src/dags/.airflowignore b/src/dags/.airflowignore
new file mode 100644
index 0000000000000000000000000000000000000000..f227618f1f9141a89dbcabacf18a6dd009506c16
--- /dev/null
+++ b/src/dags/.airflowignore
@@ -0,0 +1,3 @@
+configs
+libs
+providers
diff --git a/src/dags/default-ingest.py b/src/dags/default-ingest.py
deleted file mode 100644
index 5536623db56f379b2b2481627541b510b6cb9f63..0000000000000000000000000000000000000000
--- a/src/dags/default-ingest.py
+++ /dev/null
@@ -1,64 +0,0 @@
-#  Copyright 2020 Google LLC
-#  Copyright 2020 EPAM Systems
-#  Copyright 2020 Amazon
-#
-#  Licensed under the Apache License, Version 2.0 (the "License");
-#  you may not use this file except in compliance with the License.
-#  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-
-"""DAG for opaque ingestion."""
-
-from datetime import timedelta
-
-import airflow.utils.dates
-from airflow import DAG
-from airflow.operators.python_operator import PythonOperator
-from operators.deprecated.update_status import UpdateStatusOperator
-
-from libs.create_records import create_records  # isort:skip
-
-default_args = {
-    "owner": "Airflow",
-    "depends_on_past": False,
-    "start_date": airflow.utils.dates.days_ago(0),
-    "email": ["airflow@example.com"],
-    "email_on_failure": False,
-    "email_on_retry": False,
-    "retries": 0,
-    "retry_delay": timedelta(minutes=5),
-    "trigger_rule": "none_failed",
-}
-
-workflow_name = "Default_ingest"
-dag = DAG(
-    workflow_name,
-    default_args=default_args,
-    schedule_interval=timedelta(
-        days=1))
-
-update_status_running_op = UpdateStatusOperator(
-    task_id="update_status_running_task",
-    dag=dag
-)
-
-update_status_finished_op = UpdateStatusOperator(
-    task_id="update_status_finished_task",
-    dag=dag
-)
-
-create_records_op = PythonOperator(
-    task_id="create_records",
-    python_callable=create_records,
-    provide_context=True,
-    dag=dag
-)
-
-update_status_running_op >> create_records_op >> update_status_finished_op
diff --git a/src/dags/libs/constants.py b/src/dags/libs/constants.py
index e385809f4821e72fc3fedb78a58bdbf0106584a4..53676aa3b445a4dfb1c81736f2dc8d744eb3425a 100644
--- a/src/dags/libs/constants.py
+++ b/src/dags/libs/constants.py
@@ -18,3 +18,5 @@
 RETRIES = 3
 TIMEOUT = 1
 WAIT = 10
+
+FIRST_STORED_RECORD_INDEX = 0
diff --git a/src/dags/libs/create_records.py b/src/dags/libs/create_records.py
deleted file mode 100644
index 8e33972d0897e09152608f0b7ecdb011c5ebf9ab..0000000000000000000000000000000000000000
--- a/src/dags/libs/create_records.py
+++ /dev/null
@@ -1,104 +0,0 @@
-#  Copyright 2020 Google LLC
-#  Copyright 2020 Amazon
-#
-#  Licensed under the Apache License, Version 2.0 (the "License");
-#  you may not use this file except in compliance with the License.
-#  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-
-"""Provide R2 record processor."""
-
-import configparser
-import logging
-import os
-
-from airflow.models import Variable
-from libs.context import Context
-from libs.refresh_token import AirflowTokenRefresher
-from osdu_api.libs.auth.authorization import authorize
-from osdu_api.model.acl import Acl
-from osdu_api.model.legal.legal import Legal
-from osdu_api.model.legal.legal_compliance import LegalCompliance
-from osdu_api.model.record import Record
-from osdu_api.model.record_ancestry import RecordAncestry
-from osdu_api.storage.record_client import RecordClient
-
-logger = logging.getLogger()
-
-ACL_DICT = eval(Variable.get("acl"))
-LEGAL_DICT = eval(Variable.get("legal"))
-
-config = configparser.RawConfigParser()
-config.read(Variable.get("dataload_config_path"))
-
-DEFAULT_TENANT = config.get("DEFAULTS", "tenant")
-DEFAULT_SOURCE = config.get("DEFAULTS", "authority")
-DEFAULT_VERSION = config.get("DEFAULTS", "kind_version")
-
-
-
-@authorize(AirflowTokenRefresher())
-def create_update_record_request(headers, record_client, record):
-    """Send request to create or update records via RecordClient.
-
-    :param headers: The request headers
-    :param record_client: RecordClient
-    :param record: The record to create or update
-    :return: Storage service response
-    """
-    resp = record_client.create_update_records([record], headers)
-    return resp
-
-
-def create_records(**kwargs):
-    """Create records.
-    The only way to pass in values through the experimental api is through
-    the conf parameter
-
-    :return: response status
-    """
-    data_conf = kwargs["dag_run"].conf
-
-    logger.debug(kwargs)
-    logger.debug(data_conf)
-
-    acl = Acl(ACL_DICT["viewers"], ACL_DICT["owners"])
-    legal = Legal(
-        LEGAL_DICT["legaltags"],
-        LEGAL_DICT["otherRelevantDataCountries"],
-        LegalCompliance.compliant)
-    auth = data_conf["authorization"]
-    ancestry = RecordAncestry([])
-    record_id = None
-
-    # TODO: find out how to get kind through request not using Variables
-    kind = Variable.get("record_kind")
-    tenant, authority, file_type, version = kind.split(":")
-    total_count = {file_type.capitalize(): 1}
-    logger.info(f"The count of records to be ingested: {total_count}")
-
-    meta = [{}]
-    version = 0
-    data = data_conf.get("data", {})
-    record = Record(record_id, version, kind, acl, legal, data, ancestry, meta)
-
-    headers = {
-        "content-type": "application/json",
-        "slb-data-partition-id": data_conf.get("partition-id", DEFAULT_SOURCE),
-        "AppKey": data_conf.get("app-key", "")
-    }
-
-    record_client = RecordClient(AirflowTokenRefresher(), Context("test", "test"))
-    record_client.data_partition_id = data_conf.get(
-        "partition-id", DEFAULT_SOURCE)
-    resp = create_update_record_request(headers, record_client, record)
-    logger.info(f"Response: {resp.text}")
-    kwargs["ti"].xcom_push(key="record_ids", value=resp.json()["recordIds"])
-    return {"response_status": resp.status_code}
diff --git a/src/dags/libs/exceptions.py b/src/dags/libs/exceptions.py
index 91a307e045cc2c598ada259610e81a1696d96e02..f933923c03e631366bcaea3d48a0235ec4e7aa61 100644
--- a/src/dags/libs/exceptions.py
+++ b/src/dags/libs/exceptions.py
@@ -54,7 +54,6 @@ class FileSourceError(Exception):
 class UploadFileError(Exception):
     """Raise when there is an error while uploading a file into OSDU."""
 
-
 class TokenRefresherNotPresentError(Exception):
     """Raise when token refresher is not present in "refresh_token' decorator."""
     pass
@@ -64,6 +63,12 @@ class NoParentEntitySystemSRNError(Exception):
     """Raise when parent entity doesn't have system-generated SRN."""
     pass
 
+class NoParentEntitySystemSRNError(Exception):
+    """
+    Raise when parent entity doesn't have system-generated SRN.
+    """
+    pass
+
 
 class InvalidFileRecordData(Exception):
     """Raise when file data does not contain mandatory fields."""
diff --git a/src/dags/libs/handle_file.py b/src/dags/libs/handle_file.py
index c90d355a60e494c7eeaec6af8b89388ea20a6f1e..944fe6a2d35d2164591277e6f033de0f4c63d5d7 100644
--- a/src/dags/libs/handle_file.py
+++ b/src/dags/libs/handle_file.py
@@ -28,7 +28,7 @@ from libs.constants import RETRIES, WAIT
 from libs.context import Context
 from libs.exceptions import InvalidFileRecordData
 from libs.mixins import HeadersMixin
-from libs.refresh_token import TokenRefresher, refresh_token
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
 from providers import blob_storage
 from providers.types import BlobStorageClient, FileLikeObject
 
@@ -59,28 +59,28 @@ class FileDownloadUrlResponse:
 class FileHandler(HeadersMixin):
     """Class to perform operations using OSDU File Service."""
 
-    def __init__(self, file_service_url: str, token_refresher: TokenRefresher, context: Context,
+    def __init__(self, file_service_host: str, token_refresher: TokenRefresher, context: Context,
                  blob_storage_client: BlobStorageClient = None):
         """File handler.
 
-        :param file_service_url: Base OSDU File service url
-        :type file_service_url: str
+        :param file_service_host: Base OSDU File service url
+        :type file_service_host: str
         :param token_refresher: Object to refresh tokens
         :type token_refresher: TokenRefresher
         :param context: The tenant context data
         :type context: Context
         """
         super().__init__(context)
-        self._file_service_url = file_service_url
+        self._file_service_host = file_service_host
         self.token_refresher = token_refresher
         self._blob_storage_client = blob_storage_client or blob_storage.get_client()
 
-    def _get_file_from_preload_path(self, preload_path: str,
+    def _get_file_from_preload_path(self, preload_file_path: str,
                                     file: FileLikeObject) -> Tuple[FileLikeObject, str]:
         """Get file from a preloaded path.
 
-        :param preload_path: Full URI of the file to obtain
-        :type preload_path: str
+        :param preload_file_path: Full URI of the file to obtain
+        :type preload_file_path: str
         :return: Raw file data and content-type
         :rtype: Tuple[FileLikeObject, str]
         """
@@ -124,7 +124,7 @@ class FileHandler(HeadersMixin):
                                            kind=None)
 
     @tenacity.retry(**RETRY_SETTINGS)
-    @refresh_token()
+    @authorize()
     def _send_post_request(self, headers: dict, url: str, request_body: str) -> requests.Response:
         logger.debug(f"{request_body}")
         response = requests.post(url, request_body, headers=headers)
@@ -132,7 +132,7 @@ class FileHandler(HeadersMixin):
         return response
 
     @tenacity.retry(**RETRY_SETTINGS)
-    @refresh_token()
+    @authorize()
     def _send_get_request(self, headers: dict, url: str) -> requests.Response:
         response = requests.get(url, headers=headers)
         logger.debug(response)
@@ -148,7 +148,7 @@ class FileHandler(HeadersMixin):
         :rtype: FileUploadUrlResponse
         """
         logger.debug("Getting upload signed url.")
-        endpoint = f"{self._file_service_url}/v1/files/uploadURL"
+        endpoint = f"{self._file_service_host}/v1/files/uploadURL"
         response = self._send_get_request(headers, endpoint).json()
         logger.debug("Signed url got.")
         upload_url_response = FileUploadUrlResponse(file_id=response["FileID"],
@@ -168,7 +168,7 @@ class FileHandler(HeadersMixin):
         :rtype: FileDownloadUrlResponse
         """
         logger.debug("Getting download signed url.")
-        endpoint = f"{self._file_service_url}/v1/files/{record_id}/downloadURL"
+        endpoint = f"{self._file_service_host}/v1/files/{record_id}/downloadURL"
         response = self._send_get_request(headers, endpoint).json()
         logger.debug("Signed url got.")
         download_url_response = self._handle_download_url_response(response)
@@ -202,7 +202,7 @@ class FileHandler(HeadersMixin):
         """
         logger.debug("Getting file location.")
         request_body = json.dumps({"FileID": file_id})
-        endpoint = f"{self._file_service_url}/getFileLocation"
+        endpoint = f"{self._file_service_host}/getFileLocation"
         response = self._send_post_request(headers, endpoint, request_body)
         logger.debug("File location got.")
         return response.json()["Location"]
@@ -269,7 +269,7 @@ class FileHandler(HeadersMixin):
                 f"surrogate_name_{file_record['data']['DatasetProperties']['FileSourceInfo']['PreloadFilePath'].split('/')[-1]}"
             logger.info(f"Generated name: {file_record['data']['Name']}")
         logger.info("Sending file record metadata to File service")
-        endpoint = f"{self._file_service_url}/v1/files/metadata"
+        endpoint = f"{self._file_service_host}/v1/files/metadata"
         response = self._send_post_request(self.request_headers, endpoint, json.dumps(file_record))
 
         return response.json()["id"]
diff --git a/src/dags/libs/manifest_analyzer.py b/src/dags/libs/manifest_analyzer.py
new file mode 100644
index 0000000000000000000000000000000000000000..9bd89eb1f20976bc1d688edd916592a5c66b1aa6
--- /dev/null
+++ b/src/dags/libs/manifest_analyzer.py
@@ -0,0 +1,229 @@
+#  Copyright 2020 Google LLC
+#  Copyright 2020 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+import json
+import logging
+import re
+from collections import deque
+from typing import Set, Iterator, Iterable
+from uuid import uuid4
+
+import tenacity
+import requests
+import toposort
+from libs.constants import RETRIES, TIMEOUT
+from libs.context import Context
+from libs.mixins import HeadersMixin
+from libs.refresh_token import TokenRefresher
+from libs.traverse_manifest import ManifestEntity
+from osdu_api.libs.auth.authorization import authorize
+
+logger = logging.getLogger()
+
+
+class EntityNode(object):
+    """
+    This class represents entities and their links to parent and child ones.
+    """
+    __slots__ = ["srn", "system_srn", "entity_info", "children", "parents", "unprocessed"]
+
+    SRN_REGEX = re.compile(
+        r"(?<=\")surrogate-key:[\w\-\.\d]+(?=\")|(?<=\")[\w\-\.]+:[\w\-\.]+--[\w\-\.]+:[\w\-\.\d]+(?=\")")
+
+    def __init__(self, srn, entity_info: ManifestEntity):
+        self.srn = srn
+        self.entity_info = entity_info
+        self.system_srn = None
+        self.children = set()
+        self.parents = set()
+        self.unprocessed = False
+
+    def __repr__(self):
+        return f"SRN: {self.srn}"
+
+    @property
+    def content(self) -> dict:
+        return self.entity_info.entity
+
+    @content.setter
+    def content(self, value: dict):
+        self.entity_info.entity = value
+
+    def add_child(self, child_node: "EntityNode"):
+        self.children.add(child_node)
+
+    def add_parent(self, parent_node: "EntityNode"):
+        self.parents.add(parent_node)
+
+    def get_parent_srns(self) -> Set[str]:
+        """
+        Get list of parents' srns.
+        """
+        entity_content = json.dumps(self.content, separators=(",", ":"))
+        parent_srns = set(self.SRN_REGEX.findall(entity_content))
+        parent_srns.discard(self.srn)
+        return parent_srns
+
+    def replace_parents_surrogate_srns(self):
+        """
+        Replace surrogate parents' keys with system-generated ones in entity.
+        """
+        if not self.parents:
+            return
+        content = json.dumps(self.content)
+        for parent in self.parents:
+            if parent.system_srn:
+                content = content.replace(parent.srn, parent.system_srn)
+        self.content = json.loads(content)
+
+
+class ManifestAnalyzer(HeadersMixin):
+    """
+    This class is for creating a queue for ingesting set of data, each piece of data can depend on
+    another one, so we must prioritize the order of ingesting. The idea is to create a dependency
+    graph and traverse it to get the right order of ingestion.
+
+    The flow of prioritizing entities could be described as:
+    1. Fill graph's nodes with entities (self._fill_srn_node_table())
+    2. Create links between nodes (self._fill_nodes_edges())
+    3. Mark unprocessed nodes if they are orphaned or dependant on orphaned nodes (self._find_unprocessed_nodes())
+    4. Return prioritized queue for ingesting (self.entity_queue())
+    """
+
+    def __init__(
+        self,
+        entities: Iterable[ManifestEntity],
+        storage_service_url: str,
+        token_refresher: TokenRefresher,
+        context: Context
+    ):
+        super().__init__(context)
+        self.entities = entities
+        self.storage_service_url = storage_service_url
+        self.token_refresher = token_refresher
+        self.srn_node_table = dict()
+        self.processed_entities = []
+
+        # used as a root for all orphan entities
+        empty_entity_info = ManifestEntity("", {})
+        self.unprocessed_entities_parent = EntityNode(srn=str(uuid4()),
+                                                      entity_info=empty_entity_info)
+        self.unprocessed_entities = set()
+
+        self._fill_srn_node_table()
+        self._fill_nodes_parents()
+        self._find_unprocessed_nodes()
+
+    def _create_entity_node(self, entity: ManifestEntity):
+        srn = entity.entity.get("id", f"surrogate-key:{str(uuid4())}")
+        self.srn_node_table[srn] = EntityNode(srn, entity)
+
+    def _create_work_product_entities_nodes(self, work_product: dict):
+        for part_name, work_product_part in work_product.items():
+            if part_name == "WorkProduct":
+                self._create_entity_node(work_product_part)
+            else:
+                for p in work_product_part:
+                    self._create_entity_node(p)
+
+    def _fill_srn_node_table(self):
+        for entity in self.entities:
+            self._create_entity_node(entity)
+
+    def _fill_nodes_parents(self):
+        """
+        Find parents in every entity.
+        """
+        for entity_node in self.srn_node_table.values():
+            self._set_entity_parents(entity_node)
+
+    @tenacity.retry(
+        wait=tenacity.wait_fixed(TIMEOUT),
+        stop=tenacity.stop_after_attempt(RETRIES),
+        reraise=True
+    )
+    @authorize()
+    def _get_storage_record_request(self, headers: dict, srn: str) -> requests.Response:
+        logger.debug(f"Searching for {srn}")
+        return requests.get(f"{self.storage_service_url}/{srn}", headers=headers)
+
+    def _is_in_storage(self, parent_srn: str) -> bool:
+        try:
+            self._get_storage_record_request(self.request_headers, parent_srn)
+            return True
+        except requests.HTTPError:
+            return False
+
+    def _set_entity_parents(self, entity: EntityNode):
+        """
+        Find all references parent in entity's content.
+        If a parent is not presented in manifest, mark this entity as unprocessed.
+        If a parent is not in manifest and it is already in Storage, we do nothing with it.
+        """
+        parent_srns = entity.get_parent_srns()
+        for parent_srn in parent_srns:
+            if self.srn_node_table.get(parent_srn):
+                parent_node = self.srn_node_table[parent_srn]
+                parent_node.add_child(entity)
+                entity.add_parent(parent_node)
+            elif self._is_in_storage(parent_srn):
+                continue
+            else:  # if entity refers to srn not presenting in manifest
+                self.unprocessed_entities_parent.add_child(entity)
+                logger.info(f"'{entity}' is orphaned. Missing parent '{parent_srn}'")
+
+    def _find_unprocessed_nodes(self):
+        """
+        Traverse entities dependant on orphaned or invalid ones.
+        Add them to set of unprocessed nodes to exclude them from ingestion queue.
+        """
+        queue = deque()
+        queue.append(self.unprocessed_entities_parent)
+        while queue:
+            node = queue.popleft()
+            self.unprocessed_entities.add(node)
+            logger.debug(f"Node {node} added to unprocessed.")
+            for child in node.children:
+                if not child.unprocessed:
+                    child.unprocessed = True
+                    queue.append(child)
+        self.unprocessed_entities.discard(self.unprocessed_entities_parent)
+
+    def entity_queue(self) -> Iterator[EntityNode]:
+        """
+        Create a queue, where a child entity goes after all its parents.
+        If an entity is marked as unprocessed, then skip it.
+        """
+        entity_graph = {entity: entity.parents for entity in self.srn_node_table.values()}
+        logger.debug(f"Entity graph {entity_graph}.")
+        entity_queue = toposort.toposort_flatten(entity_graph, sort=False)
+        for entity in entity_queue:
+            if entity not in self.unprocessed_entities:
+                self.processed_entities.append(entity)
+                yield entity
+
+        for entity in self.unprocessed_entities:
+            entity.replace_parents_surrogate_srns()
+
+        logger.debug(f"Visited entities {self.processed_entities}")
+        logger.debug(f"Unprocessed entities {self.unprocessed_entities}")
+
+    def add_unprocessed_entity(self, entity: EntityNode):
+        """
+        Use if there some problems with ingesting entity.
+        Mark it and its dependants as unprocessed.
+        """
+        self.unprocessed_entities_parent.add_child(entity)
+        self._find_unprocessed_nodes()
diff --git a/src/dags/libs/process_manifest_r3.py b/src/dags/libs/process_manifest_r3.py
index 804e5ec66bc4c2d77f753b44a34331de7a025a61..86f06d26badfa550f29240ca66db21dacc36f08a 100644
--- a/src/dags/libs/process_manifest_r3.py
+++ b/src/dags/libs/process_manifest_r3.py
@@ -23,13 +23,14 @@ from typing import List, Tuple
 
 import requests
 import tenacity
-from libs.constants import RETRIES, WAIT
 from libs.context import Context
+from libs.constants import RETRIES, WAIT
 from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError
-from libs.handle_file import FileHandler
 from libs.mixins import HeadersMixin
-from libs.refresh_token import TokenRefresher, refresh_token
 from libs.source_file_check import SourceFileChecker
+from libs.handle_file import FileHandler
+from libs.traverse_manifest import ManifestEntity
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
 
 RETRY_SETTINGS = {
     "stop": tenacity.stop_after_attempt(RETRIES),
@@ -55,7 +56,6 @@ class ManifestProcessor(HeadersMixin):
     def __init__(
         self,
         storage_url: str,
-        manifest_records: List[dict],
         file_handler: FileHandler,
         source_file_checker: SourceFileChecker,
         token_refresher: TokenRefresher,
@@ -63,24 +63,21 @@ class ManifestProcessor(HeadersMixin):
     ):
         """Manifest processor.
 
-        :param storage_url: The OSDU Storage base url
-        :type storage_url: str
-        :param dagrun_conf: The conf obtained from dagrun
-        :type dagrun_conf: dict
         :param file_handler: An instance of a file handler
         :type file_handler: FileHandler
         :param source_file_checker: An instance of file checker
         :type source_file_checker: SourceFileChecker
-        :param token_refresher: An instance of token refresher
-        :type token_refresher: TokenRefresher
+        :param storage_url: The OSDU Storage base url
+        :type storage_url: str
         :param context: The tenant context
         :type context: Context
+        :param token_refresher: An instance of token refresher
+        :type token_refresher: TokenRefresher
         """
         super().__init__(context)
         self.file_handler = file_handler
         self.source_file_checker = source_file_checker
         self.storage_url = storage_url
-        self.manifest_records = manifest_records
         self.context = context
         self.token_refresher = token_refresher
 
@@ -115,6 +112,7 @@ class ManifestProcessor(HeadersMixin):
         :rtype: dict
         """
         record = copy.deepcopy(self.RECORD_TEMPLATE)
+        manifest = self._delete_surrogate_key(manifest)
         if manifest.get("id"):
             record["id"] = manifest["id"]
         record["kind"] = manifest.pop("kind")
@@ -123,6 +121,11 @@ class ManifestProcessor(HeadersMixin):
         record["data"] = manifest.pop("data")
         return record
 
+    def _delete_surrogate_key(self, entity: dict) -> dict:
+        if "surrogate-key:" in entity.get("id", ""):
+            del entity["id"]
+        return entity
+
     def _populate_file_storage_record(self, manifest: dict) -> dict:
         """Create a record from file manifest to store it via File service.
 
@@ -132,6 +135,7 @@ class ManifestProcessor(HeadersMixin):
         :rtype: dict
         """
         record = copy.deepcopy(self.RECORD_TEMPLATE)
+        manifest = self._delete_surrogate_key(manifest)
         if manifest.get("id"):
             record["id"] = manifest["id"]
         record["kind"] = manifest.pop("kind")
@@ -152,7 +156,7 @@ class ManifestProcessor(HeadersMixin):
             raise ValueError(f"Invalid answer from Storage server: {response_dict}")
 
     @tenacity.retry(**RETRY_SETTINGS)
-    @refresh_token()
+    @authorize()
     def save_record_to_storage(self, headers: dict, request_data: List[dict]) -> requests.Response:
         """
         Send request to record storage API.
@@ -197,7 +201,8 @@ class ManifestProcessor(HeadersMixin):
             if not file_record["data"]["DatasetProperties"]["FileSourceInfo"]["FileSource"]:
                 file_record = self.upload_source_file(file_record)
             else:
-                file_source = file_record["data"]["DatasetProperties"]["FileSourceInfo"]["FileSource"]
+                file_source = file_record["data"]["DatasetProperties"]["FileSourceInfo"][
+                    "FileSource"]
                 file_location = self.file_handler.get_file_staging_location(file_source)
                 self.source_file_checker.does_file_exist(file_location)
 
@@ -205,21 +210,23 @@ class ManifestProcessor(HeadersMixin):
             records.append(record)
         return records
 
-    def process_manifest(self) -> List[str]:
+    def process_manifest_records(self, manifest_records: List[ManifestEntity]) -> List[str]:
         """Process manifests and save them into Storage service.
 
+        :manifest_records: List of ManifestEntities to be ingested.
         :raises EmptyManifestError: When manifest is empty
         :return: List of ids of saved records
         :rtype: List[str]
         """
         record_ids = []
         populated_manifest_records = []
-        if not self.manifest_records:
+        if not manifest_records:
             raise EmptyManifestError
-        for manifest_record in self.manifest_records:
-            populated_manifest_records.append(self.populate_manifest_storage_record(manifest_record.get("entity")))
+        for manifest_record in manifest_records:
+            populated_manifest_records.append(
+                self.populate_manifest_storage_record(manifest_record.entity))
         save_manifests_response = self.save_record_to_storage(
-                self.request_headers, request_data=populated_manifest_records)
+            self.request_headers, request_data=populated_manifest_records)
         record_ids.extend(save_manifests_response.json()["recordIds"])
 
         return record_ids
diff --git a/tests/end-to-end-tests/test-default-ingest-success.sh b/src/dags/libs/processors/__init__.py
old mode 100755
new mode 100644
similarity index 83%
rename from tests/end-to-end-tests/test-default-ingest-success.sh
rename to src/dags/libs/processors/__init__.py
index f0908e3a3b608948df4fa3aafc5fa60ee6bf8a9a..5511adb864013540d7eda3007c4ffb6186bc68cb
--- a/tests/end-to-end-tests/test-default-ingest-success.sh
+++ b/src/dags/libs/processors/__init__.py
@@ -12,7 +12,3 @@
 #  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
-
-
-json=$(cat `dirname $0`/mock-data/default-ingest-valid.json | tail -n +15)
-airflow trigger_dag -c "$json" Default_ingest
diff --git a/src/dags/libs/processors/single_manifest_processor.py b/src/dags/libs/processors/single_manifest_processor.py
new file mode 100644
index 0000000000000000000000000000000000000000..ba2eb648b1fa191b92e1499d0b6772a7866b7462
--- /dev/null
+++ b/src/dags/libs/processors/single_manifest_processor.py
@@ -0,0 +1,117 @@
+#  Copyright 2020 Google LLC
+#  Copyright 2020 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+
+"""
+R3 Process Single Manifest helper.
+"""
+
+import logging
+from typing import List
+
+from libs.constants import FIRST_STORED_RECORD_INDEX
+from libs.context import Context
+from libs.manifest_analyzer import ManifestAnalyzer
+from libs.refresh_token import TokenRefresher
+from libs.process_manifest_r3 import ManifestProcessor
+from libs.traverse_manifest import ManifestTraversal
+from libs.validation.validate_data_integrity import DataIntegrityValidator
+from libs.validation.validate_referential_integrity import ManifestIntegrity
+from libs.validation.validate_schema import SchemaValidator
+
+logger = logging.getLogger()
+
+
+class SingleManifestProcessor(object):
+
+    def __init__(
+        self,
+        storage_url: str,
+        payload_context: Context,
+        data_integrity_validator: DataIntegrityValidator,
+        referential_integrity_validator: ManifestIntegrity,
+        manifest_processor: ManifestProcessor,
+        schema_validator: SchemaValidator,
+        token_refresher: TokenRefresher,
+    ):
+        """Init SingleManifestProcessor."""
+        self.storage_url = storage_url
+        self.payload_context = payload_context
+        self.data_integrity_validator = data_integrity_validator
+        self.referential_integrity_validator = referential_integrity_validator
+        self.manifest_processor = manifest_processor
+        self.schema_validator = schema_validator
+        self.token_refresher = token_refresher
+
+    def _process_records(self, manifest_analyzer: ManifestAnalyzer) -> str:
+        """
+        Process each entity from entity queue created according to child-parent relationships
+        between entities.
+        Replace surrogate-keys of parents inside child entities with system-generated keys.
+
+        :param manifest_analyzer: Object with proper queue of entities
+        :yield: System-generated id of entity
+        """
+        for entity in manifest_analyzer.entity_queue():
+            try:
+                logger.debug(f"Process entity {entity}")
+                entity.replace_parents_surrogate_srns()
+                record_id = self.manifest_processor.process_manifest_records(
+                    [entity.entity_info]
+                )[FIRST_STORED_RECORD_INDEX]
+                entity.system_srn = record_id
+                yield record_id
+            except Exception as e:
+                logger.warning(f"Can't process entity {entity}")
+                logger.error(e)
+                manifest_analyzer.add_unprocessed_entity(entity)
+
+    def process_manifest(self, manifest: dict, with_validation: bool) -> List[str]:
+        """Execute manifest validation then process it.
+
+        Execution steps:
+        1) validate referential manifest data integrity and remove invalid entities
+        2) validate data integrity and prune to maintain dependency consistency
+        3) initialize schema validator
+        4) validate manifest file against common schema
+        5) traverse manifest file and extract manifest entities
+        6) validate extracted manifest entities
+        7) create an ingestion queue according to child-parent relationships between entities
+        8) process valid manifest entities one-by-one
+
+        :return: List of record ids.
+        """
+        record_ids = []
+        self.data_integrity_validator.validate_manifest_data_integrity(manifest)
+        manifest_schema = self.schema_validator.validate_common_schema(manifest)
+        traversal = ManifestTraversal(manifest, manifest_schema)
+        manifest_entities = traversal.traverse_manifest()
+        logger.debug(f"entities count: {len(manifest_entities)}")
+        if with_validation:
+            manifest_entities = self.schema_validator.validate_manifest(manifest_entities)
+            logger.debug(f"valid schema entities count: {len(manifest_entities)}")
+        if with_validation:
+            self.referential_integrity_validator.ensure_integrity(manifest)
+            logger.debug(f"valid referential integrity entities count: {len(manifest_entities)}")
+        manifest_analyzer = ManifestAnalyzer(
+            manifest_entities,
+            self.storage_url,
+            self.token_refresher,
+            self.payload_context
+        )
+        for record_id in self._process_records(manifest_analyzer):
+            record_ids.append(record_id)
+        logger.info(f"Processed ids {record_ids}")
+        return record_ids
diff --git a/src/dags/libs/refresh_token.py b/src/dags/libs/refresh_token.py
index 6146da79fdbcaa3e10919f2f2f749082144b70a2..8ab038dfbf637c1853a154e5432b4efa1b762573 100644
--- a/src/dags/libs/refresh_token.py
+++ b/src/dags/libs/refresh_token.py
@@ -21,11 +21,9 @@ import os
 from abc import ABC, abstractmethod
 from functools import partial
 from http import HTTPStatus
-from typing import Any, Callable, Union
 
 import requests
-from libs.exceptions import TokenRefresherNotPresentError
-from osdu_api.libs.auth.authorization import TokenRefresher as OSDUAPITokenRefresher
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
 from providers import credentials
 from providers.types import BaseCredentials
 from tenacity import retry, stop_after_attempt
@@ -35,42 +33,7 @@ logger = logging.getLogger(__name__)
 RETRIES = 3
 
 
-class TokenRefresher(ABC):
-    """Abstract base class for token refreshers."""
-
-    @abstractmethod
-    def refresh_token(self) -> str:
-        """Refresh auth token.
-
-        :return: refreshed token
-        :rtype: str
-        """
-        pass
-
-    @property
-    @abstractmethod
-    def access_token(self) -> str:
-        """Auth access token.
-
-        :return: token string
-        :rtype: str
-        """
-        pass
-
-    @property
-    @abstractmethod
-    def authorization_header(self) -> dict:
-        """Authorization header. Must return  authorization header for
-        updating headers dict.
-        E.g. return {"Authorization": "Bearer <access_token>"}
-
-        :return: A dictionary with authorization header updated
-        :rtype: dict
-        """
-        pass
-
-
-class AirflowTokenRefresher(TokenRefresher, OSDUAPITokenRefresher):
+class AirflowTokenRefresher(TokenRefresher):
     """Simple wrapper for credentials to be used in refresh_token decorator within Airflow."""
 
     def __init__(self, creds: BaseCredentials = None):
@@ -87,7 +50,7 @@ class AirflowTokenRefresher(TokenRefresher, OSDUAPITokenRefresher):
         :rtype: str
         """
         self._credentials.refresh_token()
-        self.airflow_variables.set("access_token", self._credentials.access_token)
+        self.airflow_variables.set("core__auth__access_token", self._credentials.access_token)
         self._access_token = self._credentials.access_token
         return self._access_token
 
@@ -100,7 +63,7 @@ class AirflowTokenRefresher(TokenRefresher, OSDUAPITokenRefresher):
         """
         if not self._access_token:
             try:
-                self._access_token = self.airflow_variables.get("access_token")
+                self._access_token = self.airflow_variables.get("core__auth__access_token")
             except KeyError:
                 self.refresh_token()
         return self._access_token
@@ -113,149 +76,3 @@ class AirflowTokenRefresher(TokenRefresher, OSDUAPITokenRefresher):
         :rtype: dict
         """
         return {"Authorization": f"Bearer {self.access_token}"}
-
-
-def make_callable_request(obj: Union[object, None], request_function: Callable, headers: dict,
-                          *args, **kwargs) -> Callable:
-    """Create send_request_with_auth function.
-
-    :param obj: If wrapping a method the obj passed as first argument (self)
-    :type obj: Union[object, None]
-    :param request_function: The function to be build
-    :type request_function: Callable
-    :param headers: The request headers
-    :type headers: dict
-    :return: A partial callable
-    :rtype: Callable
-    """
-    if obj:  # if wrapped function is an object's method
-        callable_request = partial(request_function, obj, headers, *args, **kwargs)
-    else:
-        callable_request = partial(request_function, headers, *args, **kwargs)
-    return callable_request
-
-
-def _validate_headers_type(headers: dict):
-    if not isinstance(headers, dict):
-        logger.error(f"Got headers {headers}")
-        raise TypeError("Request's headers type expected to be 'dict'")
-
-
-def _validate_response_type(response: requests.Response, request_function: Callable):
-    if not isinstance(response, requests.Response):
-        logger.error(f"Function or method {request_function}"
-                     f" must return values of type 'requests.Response'. "
-                     f"Got {type(response)} instead")
-        raise TypeError
-
-
-def _validate_token_refresher_type(token_refresher: TokenRefresher):
-    if not isinstance(token_refresher, TokenRefresher):
-        raise TypeError(
-            f"Token refresher must be of type {TokenRefresher}. Got {type(token_refresher)}"
-        )
-
-
-def _get_object_token_refresher(
-    token_refresher: TokenRefresher,
-    obj: object = None
-) -> TokenRefresher:
-    """Get token refresher passed into decorator or specified in object's as
-    'token_refresher' property.
-
-    :param token_refresher: A token refresher instance
-    :type token_refresher: TokenRefresher
-    :param obj: The holder object of the decorated method, defaults to None
-    :type obj: object, optional
-    :raises TokenRefresherNotPresentError: When not found
-    :return: The token refresher
-    :rtype: TokenRefresher
-    """
-    if token_refresher is None and obj:
-        try:
-            obj.__getattribute__("token_refresher")
-        except AttributeError:
-            raise TokenRefresherNotPresentError("Token refresher must be passed into decorator or "
-                                                "set as object's 'refresh_token' attribute.")
-        else:
-            token_refresher = obj.token_refresher
-    return token_refresher
-
-
-def send_request_with_auth_header(token_refresher: TokenRefresher, *args,
-                                  **kwargs) -> requests.Response:
-    """Send request with authorization token. If response status is in
-    HTTPStatus.UNAUTHORIZED or HTTPStatus.FORBIDDEN, then refreshes token
-    and sends request once again.
-
-    :param token_refresher: The token refresher instance
-    :type token_refresher: TokenRefresher
-    :raises e: Re-raises any requests.HTTPError
-    :return: The server response
-    :rtype: requests.Response
-    """
-    obj = kwargs.pop("obj", None)
-    request_function = kwargs.pop("request_function")
-    headers = kwargs.pop("headers")
-    _validate_headers_type(headers)
-    headers.update(token_refresher.authorization_header)
-
-    send_request_with_auth = make_callable_request(obj, request_function, headers, *args, **kwargs)
-    response = send_request_with_auth()
-    _validate_response_type(response, request_function)
-
-    if not response.ok:
-        if response.status_code in (HTTPStatus.UNAUTHORIZED, HTTPStatus.FORBIDDEN):
-            token_refresher.refresh_token()
-            headers.update(token_refresher.authorization_header)
-            send_request_with_auth = make_callable_request(obj,
-                                                           request_function,
-                                                           headers,
-                                                           *args, **kwargs)
-            response = send_request_with_auth()
-        try:
-            response.raise_for_status()
-        except requests.HTTPError as e:
-            logger.error(f"{response.text}")
-            raise e
-    return response
-
-
-def refresh_token(token_refresher: TokenRefresher = None) -> Callable:
-    """Wrap a request function and check response. If response's error status
-    code is about Authorization, refresh token and invoke this function once
-    again.
-    Expects function.
-    If response is not ok and not about Authorization, then raises HTTPError
-    request_func(header: dict, *args, **kwargs) -> requests.Response
-    Or method:
-    request_method(self, header: dict, *args, **kwargs) -> requests.Response
-
-    :param token_refresher: [description], defaults to None
-    :type token_refresher: TokenRefresher, optional
-    :return: [description]
-    :rtype: Callable
-    """
-
-    def refresh_token_wrapper(request_function: Callable) -> Callable:
-        is_method = len(request_function.__qualname__.split(".")) > 1
-        if is_method:
-            def _wrapper(obj: object, headers: dict, *args, **kwargs) -> requests.Response:
-                _token_refresher = _get_object_token_refresher(token_refresher, obj)
-                _validate_token_refresher_type(_token_refresher)
-                return send_request_with_auth_header(_token_refresher,
-                                                     request_function=request_function,
-                                                     obj=obj,
-                                                     headers=headers,
-                                                     *args,
-                                                     **kwargs)
-        else:
-            def _wrapper(headers: dict, *args, **kwargs) -> requests.Response:
-                _validate_token_refresher_type(token_refresher)
-                return send_request_with_auth_header(token_refresher,
-                                                     request_function=request_function,
-                                                     headers=headers,
-                                                     *args, **kwargs)
-        return _wrapper
-
-    return refresh_token_wrapper
diff --git a/src/dags/libs/search_client.py b/src/dags/libs/search_client.py
new file mode 100644
index 0000000000000000000000000000000000000000..1c16b5d4ac24451610487c5c1d855c1c64f8498d
--- /dev/null
+++ b/src/dags/libs/search_client.py
@@ -0,0 +1,123 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+"""OSDU Search client."""
+
+import dataclasses
+import json
+import logging
+
+import requests
+import tenacity
+from libs.constants import RETRIES, WAIT
+from libs.context import Context
+from libs.mixins import HeadersMixin
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
+
+logger = logging.getLogger(__name__)
+
+TIMEOUT = 60  # seconds to wait for OSDU API response
+TENACITY_RETRY_SETTINGS = {
+    "stop": tenacity.stop_after_attempt(RETRIES),
+    "wait": tenacity.wait_fixed(WAIT),
+    "reraise": True,
+}
+
+
+@dataclasses.dataclass
+class SearchResponse:
+    """Simple class to store search results."""
+    results: list
+    total_count: int
+
+
+class SearchClient(HeadersMixin):
+    """OSDU Search Client."""
+
+    def __init__(self, search_url: str, token_refresher: TokenRefresher, context: Context):
+        """Initialize Search Client.
+
+        :param search_url: The base search url
+            (Please note that airflow var points already to the endpoint /query)
+        :type search_url: str
+        :param token_refresher: An instance of TokenRefresher
+        :type token_refresher: TokenRefresher
+        :param context: Tenant context
+        :type context: Context
+        """
+        super().__init__(context)
+        self.search_url = search_url
+        self.token_refresher = token_refresher
+
+    @tenacity.retry(**TENACITY_RETRY_SETTINGS)
+    @authorize()
+    def _send_post_request(self, headers: dict, url: str, request_body: str) -> requests.Response:
+        logger.debug(request_body)
+        response = requests.post(url, request_body, headers=headers, timeout=TIMEOUT)
+        logger.debug(response.content)
+        return response
+
+    def _create_query_request_body(self,
+                                   kind: str,
+                                   query_str: str,
+                                   limit: int,
+                                   offset: int = 0,
+                                   returned_fields: list = None,
+                                   filter_opt: dict = None) -> str:
+        """Create request body to send to Search service."""
+        request_body = {
+            "kind": kind,
+            "query": query_str,
+            "limit": limit,
+            "offset": offset,
+        }
+        if returned_fields:
+            request_body.update({"returnedFields": returned_fields})
+        if filter_opt:
+            request_body.update({"filter": filter_opt})
+        return json.dumps(request_body)
+
+    def query_records(self,
+                      kind: str,
+                      query_str: str,
+                      limit: int,
+                      offset: int = 0,
+                      returned_fields: list = None,
+                      filter_opt: dict = None) -> SearchResponse:
+        """Query records in OSDU System given parameters.
+
+        :param kind: The kind of entitities to retrieve if query matches
+        :type kind: str
+        :param query_str: An Apache Lucene compliant query string
+        :type query_str: str
+        :param limit: Number of results to return in request,
+            limit + offset < 9999
+        :type limit: int
+        :param offset: Combine with limit to paginate results, defaults to 0
+            limit + offset < 9999
+        :type offset: int, optional
+        :param returned_fields: The entity fields to return in the result list,
+            defaults to None
+        :type returned_fields: list, optional
+        :param filter_opt: An optional filter dict, defaults to None
+        :type filter_opt: dict, optional
+        :return: SearchResponse with result list and total count
+        :rtype: SearchResponse
+        """
+        request_body = self._create_query_request_body(kind, query_str, limit, offset,
+                                                       returned_fields, filter_opt)
+        response_dict = self._send_post_request(self.request_headers, self.search_url,
+                                                request_body).json()
+        return SearchResponse(results=response_dict.get("results", []),
+                              total_count=response_dict.get("totalCount", 0))
diff --git a/src/dags/libs/search_record_ids.py b/src/dags/libs/search_record_ids.py
index e45a0287ef4d3f79ffc0d90d958b0a4872eb1705..7258c9a65ada27e9a31d9b9b452f529cc5716afb 100644
--- a/src/dags/libs/search_record_ids.py
+++ b/src/dags/libs/search_record_ids.py
@@ -17,13 +17,17 @@
 
 import json
 import logging
+from typing import List
+from typing import Tuple
 
 import requests
 import tenacity
+from requests import Response
+
 from libs.context import Context
 from libs.exceptions import RecordsNotSearchableError
 from libs.mixins import HeadersMixin
-from libs.refresh_token import TokenRefresher, refresh_token
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
 
 logger = logging.getLogger()
 
@@ -103,7 +107,7 @@ class SearchId(HeadersMixin):
         stop=tenacity.stop_after_attempt(RETRIES),
         reraise=True
     )
-    @refresh_token()
+    @authorize()
     def search_files(self, headers: dict) -> requests.Response:
         """Send request with recordIds to Search service.
 
@@ -130,3 +134,54 @@ class SearchId(HeadersMixin):
         """Check if every record in self.record_ids is searchable."""
         headers = self.request_headers
         self.search_files(headers)
+
+
+class ExtendedSearchId(SearchId):
+
+    def __init__(self, search_url: str, record_ids: list, token_refresher, context: Context):
+        super().__init__(search_url, record_ids, token_refresher, context)
+
+    def _create_request_body(self):
+        """
+        Create request body to send to Search service.
+        """
+        query = self._create_search_query()
+        request_body = {
+            "kind": "*:*:*:*",
+            "query": query,
+            "returnedFields": ["id", "version"]
+        }
+        self.request_body = json.dumps(request_body)
+
+    def _extract_id_from_response(self, response: dict):
+        results = response.get("results")
+        record_ids = [":".join([elem.get("id"), str(elem.get("version", ""))]) for elem in results]
+        record_ids.extend([elem.get("id") for elem in results])
+        logger.debug(f"response ids: {record_ids}")
+        return record_ids
+
+    @authorize()
+    def _make_post_request(self, headers: dict, request_body: dict) -> Response:
+        return requests.post(self.search_url, request_body, headers=headers)
+
+    def search_records(self) -> Tuple[List, List]:
+        """
+        Send request with recordIds to Search service.
+        """
+        if self.request_body:
+            response = self._make_post_request(self.request_headers, self.request_body)
+            logger.debug(response.text)
+            data = response.json()
+            total_count = data.get('totalCount')
+            logger.debug(f"Got total count {total_count}")
+            if total_count is None:
+                raise ValueError(f"Got no totalCount field in Search service response. "
+                                 f"Response is {data}.")
+            response_records_ids = set(self._extract_id_from_response(data))
+            if set(self.record_ids).issubset(response_records_ids):
+                return self.record_ids, []
+            else:
+                found_ids = list(set(self.record_ids).intersection(response_records_ids))
+                missing_ids = list(set(self.record_ids).difference(response_records_ids))
+                logger.warning(f"The next ids are absent in the system: {missing_ids}")
+                return found_ids, missing_ids
diff --git a/src/dags/libs/traverse_manifest.py b/src/dags/libs/traverse_manifest.py
index af56b709427180f956057ec7698b533eb9a3a69b..4aaf49f1a1d91549fa6041b4c43cbf7193109648 100644
--- a/src/dags/libs/traverse_manifest.py
+++ b/src/dags/libs/traverse_manifest.py
@@ -17,17 +17,28 @@ import copy
 import logging
 from typing import List
 
+import dataclasses
 from libs.exceptions import EmptyManifestError
 
 
 logger = logging.getLogger()
 
 
+@dataclasses.dataclass()
+class ManifestEntity:
+    schema: str
+    entity: dict
+
+    def __eq__(self, other: "ManifestEntity"):
+        return self.entity == other.entity\
+               and self.schema == other.schema
+
+
 class ManifestTraversal(object):
     """Class to traverse manifest and extract all manifest records"""
 
-    def __init__(self, dagrun_conf: dict, manifest_schema: dict):
-        self.data_object = copy.deepcopy(dagrun_conf)
+    def __init__(self, manifest: dict, manifest_schema: dict):
+        self.manifest = copy.deepcopy(manifest)
         self.manifest_schema = manifest_schema
         self.manifest_info = []
 
@@ -40,11 +51,7 @@ class ManifestTraversal(object):
         :return:
         """
         extracted_schema = schema.split("/")[-1]
-        logger.debug(f"Extracted schema kind: {extracted_schema}")
-        return {
-            "schema": extracted_schema,
-            "entity": entity
-        }
+        return ManifestEntity(entity=entity, schema=extracted_schema)
 
     def _traverse_list(self, manifest_entities: List[dict], property_name: str, manifest_schema_part: dict):
         """
@@ -57,36 +64,34 @@ class ManifestTraversal(object):
                                                manifest_schema_part[property_name]["items"]["$ref"]))
         return entities
 
-    def traverse_manifest(self) -> List[dict]:
+    def traverse_manifest(self) -> List[ManifestEntity]:
         """
         Traverse manifest structure and return the list of manifest records.
 
         :return: list of records
         """
-        if "manifest" not in self.data_object:
+        if not self.manifest:
             raise EmptyManifestError
-
         manifest_entities = []
-        manifest_file = self.data_object["manifest"]
-        if manifest_file.get("ReferenceData"):
-            manifest_entities.extend(self._traverse_list(manifest_file["ReferenceData"],
+        if self.manifest.get("ReferenceData"):
+            manifest_entities.extend(self._traverse_list(self.manifest["ReferenceData"],
                                                          "ReferenceData",
                                                          self.manifest_schema["properties"]))
-        if manifest_file.get("MasterData"):
-            manifest_entities.extend(self._traverse_list(manifest_file["MasterData"],
+        if self.manifest.get("MasterData"):
+            manifest_entities.extend(self._traverse_list(self.manifest["MasterData"],
                                                          "MasterData",
                                                          self.manifest_schema["properties"]))
-        if manifest_file.get("Data"):
-            if manifest_file["Data"].get("WorkProduct"):
+        if self.manifest.get("Data"):
+            if self.manifest["Data"].get("WorkProduct"):
                 manifest_entities.append(self._populate_manifest_entity(
-                        manifest_file["Data"]["WorkProduct"],
+                        self.manifest["Data"]["WorkProduct"],
                         self.manifest_schema["properties"]["Data"]["properties"]["WorkProduct"]["$ref"]))
-            if manifest_file["Data"].get("WorkProductComponents"):
-                manifest_entities.extend(self._traverse_list(manifest_file["Data"]["WorkProductComponents"],
+            if self.manifest["Data"].get("WorkProductComponents"):
+                manifest_entities.extend(self._traverse_list(self.manifest["Data"]["WorkProductComponents"],
                                                              "WorkProductComponents",
                                                              self.manifest_schema["properties"]["Data"]["properties"]))
-            if manifest_file["Data"].get("Datasets"):
-                manifest_entities.extend(self._traverse_list(manifest_file["Data"]["Datasets"],
+            if self.manifest["Data"].get("Datasets"):
+                manifest_entities.extend(self._traverse_list(self.manifest["Data"]["Datasets"],
                                                              "Datasets",
                                                              self.manifest_schema["properties"]["Data"]["properties"]))
         return manifest_entities
diff --git a/tests/end-to-end-tests/test-default-ingest-fail.sh b/src/dags/libs/types.py
old mode 100755
new mode 100644
similarity index 75%
rename from tests/end-to-end-tests/test-default-ingest-fail.sh
rename to src/dags/libs/types.py
index 4f1cb6fb6106190b726f35ee4d2adf2e666f21f0..ffc4921c83690e6687451d09948996b67c7413ac
--- a/tests/end-to-end-tests/test-default-ingest-fail.sh
+++ b/src/dags/libs/types.py
@@ -1,5 +1,5 @@
-#  Copyright 2020 Google LLC
-#  Copyright 2020 EPAM Systems
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
 #
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.
@@ -13,5 +13,6 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-json=$(cat `dirname $0`/mock-data/default-ingest-invalid.json | tail -n +15)
-airflow trigger_dag -c "$json" Default_ingest
+from typing import TypeVar, List
+
+ManifestType = TypeVar("ManifestType", dict, List[dict])
diff --git a/src/dags/libs/update_status.py b/src/dags/libs/update_status.py
index d6ac93503b4d2cc7537dff3816d94e2de9a63284..afc0d4f3bee4ed9af730dbe0d238c101b3d057cb 100644
--- a/src/dags/libs/update_status.py
+++ b/src/dags/libs/update_status.py
@@ -22,7 +22,7 @@ import requests
 
 from libs.context import Context
 from libs.mixins import HeadersMixin
-from libs.refresh_token import TokenRefresher, refresh_token
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
 
 logger = logging.getLogger()
 
@@ -66,7 +66,7 @@ class UpdateStatus(HeadersMixin):
         self.status = status
         self.token_refresher = token_refresher
 
-    @refresh_token()
+    @authorize()
     def update_status_request(self, headers: dict) -> requests.Response:
         """Send request to update status.
 
@@ -85,7 +85,7 @@ class UpdateStatus(HeadersMixin):
         response = requests.put(update_status_url, request_body, headers=headers)
         return response
 
-    @refresh_token()
+    @authorize()
     def update_status_request_old(self, headers: dict) -> requests.Response:
         """Send request to update status.
 
diff --git a/tests/end-to-end-tests/mock-data/default-ingest-invalid.json b/src/dags/libs/validation/__init__.py
similarity index 63%
rename from tests/end-to-end-tests/mock-data/default-ingest-invalid.json
rename to src/dags/libs/validation/__init__.py
index 5e8c414090ca85c41fec20b93c07f9914ce57cae..de8f5ce5bf56bdd78824065dbbd52846984960b1 100644
--- a/tests/end-to-end-tests/mock-data/default-ingest-invalid.json
+++ b/src/dags/libs/validation/__init__.py
@@ -1,5 +1,5 @@
-#  Copyright 2020 Google LLC
-#  Copyright 2020 EPAM Systems
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
 #
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.
@@ -12,11 +12,3 @@
 #  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
-
-{
-    "authorization": "Bearer secret",
-    "legal-tags": "\"legaltags\": [\"foo\"],\"otherRelevantDataCountries\": [\"FR\",\"US\",\"CA\"]\"",
-    "AppKey": "test",
-    "acl": "\"viewers\": [\"foo\"], \"owners\": [\"foo\"]",
-    "data-partition-id": "odes"
-}
diff --git a/src/dags/libs/validation/validate_data_integrity.py b/src/dags/libs/validation/validate_data_integrity.py
new file mode 100644
index 0000000000000000000000000000000000000000..9261f11dcd4d4d3cf2729bbc971ffe676641aa6a
--- /dev/null
+++ b/src/dags/libs/validation/validate_data_integrity.py
@@ -0,0 +1,175 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+"""Module for validation of WorkProduct, WorkProductComponents and Datasets."""
+
+import functools
+import json
+import logging
+from typing import Any, Dict, Iterable, Set, Tuple
+
+from libs.search_client import SearchClient
+from libs.validation.validate_file_source import FileSourceValidator
+
+logger = logging.getLogger(__name__)
+
+
+class DataIntegrityValidator:
+    """Provides functions to validate Manifest["Data"]."""
+
+    def __init__(self, search_client: SearchClient, file_source_validator: FileSourceValidator):
+        """Initialize validator with dependencies."""
+        self._search_client = search_client
+        self._file_source_validator = file_source_validator
+
+    @staticmethod
+    def _collect_ids_from_object_array(object_array: Iterable[Dict[str, Any]]) -> Set[str]:
+        """Collects ids from an object array."""
+        ids_set = set(map(lambda obj: obj.get("id"), object_array))
+        ids_set.discard(None)
+        return ids_set
+
+    @staticmethod
+    def _create_search_ids_query_str(ids_list) -> str:
+        """Create an Apache Lucene compliant query."""
+        ids_list_str = " OR ".join(f"\"{_id}\"" for _id in ids_list)
+        query = f"id:({ids_list_str})"
+        logger.debug(query)
+        return query
+
+    def _search_for_entities(self, ids_list: Iterable[str]) -> Iterable[str]:
+        """Returns entities ids found in OSDU system."""
+        query_str = self._create_search_ids_query_str(ids_list)
+        all_kinds = "*:*:*:*"
+        search_response = self._search_client.query_records(kind=all_kinds,
+                                                            query_str=query_str,
+                                                            limit=len(ids_list),
+                                                            returned_fields=["id"])
+        return [result.get("id") for result in search_response.results]
+
+    def _update_ids_from_search(self, expected_ids_set, all_ids_set):
+        """Updates all_ids_set with a query to OSDU system."""
+        ids_to_search = list(filter(lambda _id: "surrogate" not in _id, expected_ids_set))
+        if ids_to_search:
+            logger.debug(f"ids_to_search {ids_to_search}")
+            ids_found = self._search_for_entities(ids_to_search)
+            logger.debug(f"ids_found: {ids_found}")
+            all_ids_set.update(ids_found)
+
+    def _validate_wpcs_to_datasets(
+            self, work_product_components: Iterable[dict],
+            datasets: Iterable[dict]) -> Tuple[Iterable[dict], Iterable[dict]]:
+        """Validate that all Datasets referenced from any WorkProductComponent
+        exist and validate there are no orphan Datasets.
+
+        :param work_product_components: A list of WorkProductComponents
+        :type work_product_components: Iterable[dict]
+        :param datasets: A list of Datasets
+        :type datasets: Iterable[dict]
+        :return: A tuple containing (valid_work_product_components, valid_datasets)
+        :rtype: Tuple[Iterable[dict], Iterable[dict]]
+        """
+        all_datasets_ids = self._collect_ids_from_object_array(datasets)
+        all_expected_datasets_ids = set(
+            functools.reduce(lambda a, b: a + b,
+                             (wpc["data"]["Datasets"] for wpc in work_product_components)))
+        self._update_ids_from_search(all_expected_datasets_ids, all_datasets_ids)
+
+        all_valid_datasets_ids = set()
+        filtered_wpcs = []
+        for wpc in work_product_components:
+            expected_datasets_ids = set(wpc["data"]["Datasets"])
+            valid_datasets_ids = expected_datasets_ids.intersection(all_datasets_ids)
+            all_valid_datasets_ids.update(valid_datasets_ids)
+            diffs = expected_datasets_ids.symmetric_difference(valid_datasets_ids)
+            if not diffs:
+                filtered_wpcs.append(wpc)
+            else:
+                wpc_id = wpc.get("id")
+                logger.error(f"Rejecting WorkProductComponent with id: {wpc_id}.\n"
+                             f"Reason: following datasets ids don't match: {diffs}")
+
+        filtered_datasets = []
+        for dataset in datasets:
+            dataset_id = dataset.get("id")
+            if dataset_id in all_valid_datasets_ids:
+                filtered_datasets.append(dataset)
+            else:
+                logger.error(f"Rejecting orphan dataset with id: {dataset_id}")
+
+        return filtered_wpcs, filtered_datasets
+
+    def _validate_wp_to_wpcs(self, work_product: Dict[str, Any],
+                             work_product_components: Iterable[dict]) -> Dict[str, Any]:
+        """Validate all WorkProductComponents referenced from WorkProduct exist.
+
+        :param work_product: The WorkProduct metadata object
+        :type work_product: Dict[str, Any]
+        :param work_product_components: A list of WorkProductComponents
+        :type work_product_components: Iterable[dict]
+        :return: Validated WorkProduct or empty if invalid.
+        :rtype: Dict[str, Any]
+        """
+        if not work_product:
+            return {}
+        all_wpcs_ids = self._collect_ids_from_object_array(work_product_components)
+        expected_wpc_ids = set(work_product["data"]["Components"])
+        self._update_ids_from_search(expected_wpc_ids, all_wpcs_ids)
+        diffs = expected_wpc_ids.symmetric_difference(all_wpcs_ids)
+        if not diffs:
+            return work_product
+        else:
+            logger.error(f"Rejecting WorkProduct and all its entities: WPC and Datasets.\n"
+                         f"Reason: following work_product_components ids are inconsistent: {diffs}")
+            return {}
+
+    def _validate_manifest_data_datasets(self, manifest_data: Dict[str, Any]):
+        """Delegate call to FileSource validator to validate datasets.
+
+        :param manifest_data: `Data` fragment in Manifest 1.0.0
+        :type manifest_data: Dict[str, Any]
+        """
+        manifest_data_datasets = manifest_data.get("Datasets")
+        if not manifest_data_datasets:
+            logger.debug("No datasets found. Skipping validation.")
+            return
+        valid_datasets = self._file_source_validator.filter_valid_datasets(manifest_data_datasets)
+        manifest_data["Datasets"] = valid_datasets
+
+    def validate_manifest_data_integrity(self, manifest: Dict[str, Any]):
+        """Validate `Data` field in Manifest.
+
+        :param manifest: Manifest 1.0.0
+        :type manifest: Dict[str: Any]
+        """
+        manifest_data = manifest.get("Data")
+        if not manifest_data:
+            logger.debug("No Data found, skipping validation")
+            return
+
+        self._validate_manifest_data_datasets(manifest_data)
+
+        if manifest_data.get("WorkProductComponents") and manifest_data.get("Datasets"):
+            valid_wpcs, valid_datasets = self._validate_wpcs_to_datasets(
+                manifest_data["WorkProductComponents"], manifest_data["Datasets"])
+            valid_wp = self._validate_wp_to_wpcs(manifest_data["WorkProduct"], valid_wpcs)
+
+            if valid_wp:
+                manifest_data["WorkProduct"] = valid_wp
+                manifest_data["WorkProductComponents"] = valid_wpcs
+                manifest_data["Datasets"] = valid_datasets
+            else:
+                manifest["Data"] = {}
+        else:
+            manifest["Data"] = {}
diff --git a/src/dags/libs/validation/validate_file_source.py b/src/dags/libs/validation/validate_file_source.py
new file mode 100644
index 0000000000000000000000000000000000000000..90199fc2878a46b68e43f55173ecd6f2b830873c
--- /dev/null
+++ b/src/dags/libs/validation/validate_file_source.py
@@ -0,0 +1,110 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+"""Module for validation of File and FileCollection before ingesting."""
+
+import logging
+from typing import Any, Dict, List
+
+logger = logging.getLogger(__name__)
+
+
+class DatasetType:
+    FILE = ":dataset--File."
+    FILE_COLLECTION = ":dataset--FileCollection."
+
+
+class FileSourceValidator:
+    """Validates FileSource or IndexFilePath according to the provided type."""
+    def _validate_file_source_info(self, file_source_info: dict) -> bool:
+        """Validates file source info is populated correctly.
+
+        :param file_source_info: An instance of AbstractFileSourceInfo 1.0.0
+        :type file_source_info: dict
+        :return: True if validation succeed or field was populated.
+        :rtype: bool
+        """
+        if file_source_info["FileSource"]:
+            return True
+        else:
+            return False
+
+    def _validate_file_record(self, file_record: dict) -> bool:
+        """Validate file source info in file record.
+
+        :param file_record: File record metadata
+        :type file_record: dict
+        :return: True if file record has FileSource info, False otherwise
+        :rtype: bool
+        """
+        return self._validate_file_source_info(
+            file_record["data"]["DatasetProperties"]["FileSourceInfo"])
+
+    def _validate_file_collection_record(self, file_collection_record: dict) -> bool:
+        """Validate file source info in file collection record
+
+        :param file_collection_record: FileCollection record metadata
+        :type file_collection_record: dict
+        :return: True if file collection record has IndexFilePath info or all
+            FileSourceInfo from inner elements, False otherwise
+        :rtype: bool
+        """
+        index_file_path = file_collection_record["data"]["DatasetProperties"].get("IndexFilePath")
+        if index_file_path:
+            return True
+        else:
+            file_source_infos = []
+            for file_source_info in file_collection_record["data"]["DatasetProperties"].get(
+                    "FileSourceInfos", []):
+                file_source_infos.append(self._validate_file_source_info(file_source_info))
+            return bool(file_source_infos) and all(file_source_infos)
+
+    def filter_valid_datasets(self, datasets: List[dict]) -> List[dict]:
+        """Filter only valid (which contains FileSource info) datasets
+
+        :param datasets: A list of {File, FileCollection} datasets
+        :type datasets: List[dict]
+        :return: The filtered list of valid {File, FileCollection} datasets
+        :rtype: List[dict]
+        """
+        valid_datasets = []
+        for dataset in datasets:
+            is_file = DatasetType.FILE in dataset.get("kind", "")
+            is_file_collection = DatasetType.FILE_COLLECTION in dataset.get("kind", "")
+            is_valid_dataset = False
+
+            if is_file:
+                try:
+                    is_valid_dataset = self._validate_file_record(dataset)
+                except KeyError as exc:
+                    logger.error(
+                        f"Rejecting invalid dataset: {dataset.get('id')}, invalid structure. KeyError: {exc}"
+                    )
+            elif is_file_collection:
+                try:
+                    is_valid_dataset = self._validate_file_collection_record(dataset)
+                except KeyError as exc:
+                    logger.error(
+                        f"Rejecting invalid dataset: {dataset.get('id')}, invalid structure. KeyError: {exc}"
+                    )
+            else:
+                logger.error(f"Unknown file type: {dataset.get('kind')}.")
+
+            if is_valid_dataset:
+                valid_datasets.append(dataset)
+            else:
+                logger.error(
+                    f"Rejecting dataset: {dataset.get('id')}. Reason: No FileSource info found.")
+
+        return valid_datasets
diff --git a/src/dags/libs/validation/validate_referential_integrity.py b/src/dags/libs/validation/validate_referential_integrity.py
new file mode 100644
index 0000000000000000000000000000000000000000..1c03a29dfc7bc5973538a0bc584bcb2ffb01e9c0
--- /dev/null
+++ b/src/dags/libs/validation/validate_referential_integrity.py
@@ -0,0 +1,169 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+import copy
+import json
+import re
+import logging
+from typing import List, re as regex
+
+from libs.context import Context
+from libs.exceptions import EmptyManifestError
+from libs.search_record_ids import ExtendedSearchId
+
+logger = logging.getLogger()
+
+
+class ManifestIntegrity(object):
+    """Class to validate if parents reference and master data are exists and
+    remove non-valid entities to provide integrity
+    """
+    REFERENCE_DATA_ID_PATTERN = re.compile(r"[\w\-\.]+:reference-data\-\-[\w\-\.]+:.[^,;\"]+",
+                                           re.I + re.M)
+    MASTER_DATA_ID_PATTERN = re.compile(r"[\w\-\.]+:master-data\-\-[\w\-\.]+:.[^,;\"]+",
+                                        re.I + re.M)
+    WORK_PRODUCT_ID_PATTERN = re.compile(r"[\w\-\.]+:work-product\-\-[\w\-\.]+:.[^,;\"]+",
+                                         re.I + re.M)
+    WORK_PRODUCT_COMPONENT_ID_PATTERN = re.compile(
+        r"[\w\-\.]+:work-product-component\-\-[\w\-\.]+:.[^,;\"]+", re.I + re.M)
+    DATASET_ID_PATTERN = re.compile(r"[\w\-\.]+:dataset\-\-[\w\-\.]+:.[^,;\"]+", re.I + re.M)
+
+    def __init__(self, search_url: str, token_refresher, context: Context):
+        self.search_url = search_url
+        self.token_refresher = token_refresher
+        self.context = context
+        self.ids_for_validation = []
+        self.entities_ids = {}
+        self.patterns = [self.REFERENCE_DATA_ID_PATTERN,
+                         self.MASTER_DATA_ID_PATTERN,
+                         self.WORK_PRODUCT_ID_PATTERN,
+                         self.WORK_PRODUCT_COMPONENT_ID_PATTERN,
+                         self.DATASET_ID_PATTERN]
+
+    def _match_id_with_pattern(self, pattern: regex.Pattern, source: str) -> List[str]:
+        return pattern.findall(source)
+
+    def _collect_ids(self, manifest_file: dict):
+        """
+        Collect manifest entities ids to exclude them while integrity checking
+        """
+        entities_ids = []
+        if manifest_file.get("ReferenceData"):
+            for elem in manifest_file["ReferenceData"]:
+                if elem.get("id"):
+                    entities_ids.append(elem["id"])
+        if manifest_file.get("MasterData"):
+            for elem in manifest_file["MasterData"]:
+                if elem.get("id"):
+                    entities_ids.append(elem["id"])
+        if manifest_file.get("Data"):
+            if manifest_file["Data"].get("WorkProduct") and manifest_file["Data"]["WorkProduct"].get("id"):
+                entities_ids.append(manifest_file["Data"]["WorkProduct"]["id"])
+            if manifest_file["Data"].get("WorkProductComponents"):
+                for elem in manifest_file["Data"]["WorkProductComponents"]:
+                    if elem.get("id"):
+                        entities_ids.append(elem["id"])
+            if manifest_file["Data"].get("Datasets"):
+                for elem in manifest_file["Data"]["Datasets"]:
+                    if elem.get("id"):
+                        entities_ids.append(elem["id"])
+        self.entities_ids = set(entities_ids)
+
+    def _remove_redundant_colon(self, ids: List[str]) -> List[str]:
+        """
+        Remove symbol ':' from reference ids without versions
+        """
+        cleaned_ids = []
+        for elem in ids:
+            if elem.endswith(":"):
+                cleaned_ids.append(elem[:-1])
+            else:
+                cleaned_ids.append(elem)
+        return cleaned_ids
+
+    def _extract_references(self, manifest: dict) -> List[str]:
+        """
+        Extract reference ids from manifest
+        """
+        temp_manifest_entity = copy.deepcopy(manifest)
+        temp_manifest_entity.pop("kind")
+        manifest_str = json.dumps(temp_manifest_entity)
+        ids_for_validation = []
+        for pattern in self.patterns:
+            ids_for_validation.extend(self._match_id_with_pattern(pattern, manifest_str))
+        ids_for_validation = list(set(ids_for_validation).difference(self.entities_ids))
+        cleaned_ids_for_validation = self._remove_redundant_colon(ids_for_validation)
+        logger.debug(f"Extracted reference ids: {cleaned_ids_for_validation}")
+        return cleaned_ids_for_validation
+
+    def _validate_entity(self, manifest: dict) -> bool:
+        """
+        Validate reference ids from manifest entity
+        """
+        ids_for_validation = self._extract_references(manifest)
+        missing_ids = None
+        if ids_for_validation:
+            search_class = ExtendedSearchId(self.search_url, ids_for_validation, self.token_refresher, self.context)
+            found_ids, missing_ids = search_class.search_records()
+        if not missing_ids:
+            return True
+        return False
+
+    def ensure_integrity(self, manifest_file: dict=None):
+        """
+        Validate reference ids in traversal manifest file
+
+        :return: only valid entities in the same structure
+        """
+
+        if not manifest_file:
+            raise EmptyManifestError()
+
+        self._collect_ids(manifest_file)
+
+        if manifest_file.get("ReferenceData"):
+            for data in manifest_file["ReferenceData"]:
+                is_valid = self._validate_entity(data)
+                if not is_valid:
+                    logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                    manifest_file["ReferenceData"].remove(data)
+        if manifest_file.get("MasterData"):
+            master_data = manifest_file["MasterData"]
+            for data in master_data:
+                is_valid = self._validate_entity(data)
+                if not is_valid:
+                    logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                    manifest_file["MasterData"].remove(data)
+        if manifest_file.get("Data"):
+            if manifest_file["Data"].get("WorkProduct"):
+                work_product_data = manifest_file["Data"]["WorkProduct"]
+                is_valid = self._validate_entity(work_product_data)
+                if not is_valid:
+                    logger.warning(f"Resource with kind {work_product_data.get('kind')} was rejected")
+                    manifest_file["Data"]["WorkProduct"] = {}
+            if manifest_file["Data"].get("WorkProductComponents"):
+                wpc_data = manifest_file["Data"]["WorkProductComponents"]
+                for data in wpc_data:
+                    is_valid = self._validate_entity(data)
+                    if not is_valid:
+                        logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                        manifest_file["Data"]["WorkProductComponents"].remove(data)
+            if manifest_file["Data"].get("Datasets"):
+                datasets = manifest_file["Data"]["Datasets"]
+                for data in datasets:
+                    is_valid = self._validate_entity(data)
+                    if not is_valid:
+                        logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                        manifest_file["Data"]["Datasets"].remove(data)
diff --git a/src/dags/libs/validate_schema.py b/src/dags/libs/validation/validate_schema.py
similarity index 68%
rename from src/dags/libs/validate_schema.py
rename to src/dags/libs/validation/validate_schema.py
index d51c557d8229436cdef508d1de5347d08d22e5ff..c54271ce1f1b06a031e8b04b306e154ecae353f7 100644
--- a/src/dags/libs/validate_schema.py
+++ b/src/dags/libs/validation/validate_schema.py
@@ -16,9 +16,8 @@
 """Provides SchemaValidator."""
 
 import copy
-import json
 import logging
-from typing import Union, Any, List
+from typing import Any, List, Union
 
 import jsonschema
 import requests
@@ -26,8 +25,9 @@ import tenacity
 from jsonschema import exceptions
 from libs.context import Context
 from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError
+from libs.traverse_manifest import ManifestEntity
 from libs.mixins import HeadersMixin
-from libs.refresh_token import TokenRefresher, refresh_token
+from osdu_api.libs.auth.authorization import TokenRefresher, authorize
 
 logger = logging.getLogger()
 
@@ -74,20 +74,16 @@ class SchemaValidator(HeadersMixin):
     """Class to validate schema of Manifests."""
 
     def __init__(
-        self, schema_service: str,
+        self,
+        schema_service: str,
         token_refresher: TokenRefresher,
         context: Context
     ):
         """Init SchemaValidator.
 
         :param schema_service: The base OSDU Schema service url
-        :type schema_service: str
-        :param dagrun_conf: The airflow dagrun.conf
-        :type dagrun_conf: dict
         :param token_refresher: An instance of token refresher
-        :type token_refresher: TokenRefresher
         :param context: The tenant context
-        :type context: Context
         """
         super().__init__(context)
         self.schema_service = schema_service
@@ -104,28 +100,25 @@ class SchemaValidator(HeadersMixin):
         stop=tenacity.stop_after_attempt(RETRIES),
         reraise=True
     )
-    @refresh_token()
+    @authorize()
     def _get_schema_from_schema_service(self, headers: dict, uri: str) -> requests.Response:
         """Send request to Schema service to retrieve schema."""
         response = requests.get(uri, headers=headers, timeout=60)
         return response
 
-    def __delete_refs(self, schema_part: Union[dict, list]):
+    def _clear_data_fields(self, schema_part: Union[dict, list]):
         """
-        Recursively clear a schema's object parts containing "$ref".
+        Clear a schema's ReferenceData, Data and MasterData fields".
         This method is used by generic manifest validation, deleting these fields make such a
         validation.more generic.
         :param schema_part:
         """
-        if isinstance(schema_part, dict):
-            if "$ref" in schema_part:
-                schema_part.clear()
-            else:
-                for k in schema_part:
-                    self.__delete_refs(schema_part[k])
-        elif isinstance(schema_part, list):
-            for i in schema_part:
-                self.__delete_refs(i)
+        if schema_part.get("ReferenceData"):
+            schema_part["ReferenceData"] = {}
+        if schema_part.get("Data"):
+            schema_part["Data"] = {}
+        if schema_part.get("MasterData"):
+            schema_part["MasterData"] = {}
 
     def get_schema_request(self, uri: str) -> dict:
         """Get schema from Schema service. Change $id field to url.
@@ -164,9 +157,8 @@ class SchemaValidator(HeadersMixin):
         """
         if not schema:
             schema = self.get_schema(entity["kind"])
-        data = entity
         try:
-            self._validate_against_schema(schema, data)
+            self._validate_against_schema(schema, entity)
             logger.debug(f"Record successfully validated")
             return True
         except exceptions.ValidationError as exc:
@@ -175,17 +167,6 @@ class SchemaValidator(HeadersMixin):
             logger.error(f"Error: {exc}")
             return False
 
-    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_entity(component)
-            else:
-                self._validate_entity(value)
-
     def _validate_against_schema(self, schema: dict, data: Any):
         """
         Validate any data against schema.
@@ -203,27 +184,6 @@ class SchemaValidator(HeadersMixin):
         validator = jsonschema.Draft7Validator(schema=schema, resolver=resolver)
         validator.validate(data)
 
-    def _validate_data_group(self, entities: list):
-        """
-        Validate each entity from a list of entities.
-        :param entities:
-        :return:
-        """
-        if isinstance(entities, list):
-            for entity in entities:
-                self._validate_entity(entity)
-
-    def _validate_whole_manifest(self, manifest: dict):
-        """
-        Validate any manifest in general.
-        Also at this step verify that MasterData, ReferenceData, WorkProduct, WorkProductComponents,
-        Files entities correspond their generic schemas, because references to those schemas are in
-        a Manifest schema.
-        """
-        schema = self.get_schema(manifest["kind"])
-        logger.debug(f"Validating kind {manifest['kind']}")
-        self._validate_against_schema(schema, manifest)
-
     def validate_common_schema(self, manifest: dict) -> dict:
         """
         This is a preliminary validation of a manifest that verifies that a manifest corresponds
@@ -232,46 +192,71 @@ class SchemaValidator(HeadersMixin):
         :param manifest:
         :return: Manifest schema
         """
-        if "manifest" not in manifest:
-            raise EmptyManifestError
-        schema = self.get_schema(manifest["manifest"]["kind"])
+        schema = self.get_schema(manifest["kind"])
         schema_without_refs = copy.deepcopy(schema)
         if schema_without_refs.get("properties"):
-            self.__delete_refs(schema_without_refs["properties"])
+            self._clear_data_fields(schema_without_refs["properties"])
         else:
-            self.__delete_refs(schema_without_refs)
+            self._clear_data_fields(schema_without_refs)
         logger.debug("Schema without refs")
         logger.debug(f"{schema_without_refs}")
-        self._validate_against_schema(schema, manifest)
+        self._validate_against_schema(schema_without_refs, manifest)
         return schema
 
-    def _validate_against_generic_schema(self, schema: str, entity: Any) -> bool:
-        try:
-            self._validate_against_schema(schema, entity)
-            logger.debug(f"Record successfully validated against generic schema.")
-            return True
-        except exceptions.ValidationError as exc:
-            logger.error("Schema validation error.")
-            logger.error(f"Manifest kind: {entity['kind']}")
-            logger.error(f"Manifest: {entity}")
-            logger.error(f"Error: {exc}")
-            return False
-
-    def validate_manifest(self, manifest_records: List[dict]) -> List[dict]:
+    def validate_manifest(self, manifest_records: List[ManifestEntity]) -> List[ManifestEntity]:
         """
-        Validate manifest. Raise error if manifest is not valid.
+        Validate manifest's entities one-by-one. Return list of
+        :param manifest_records: List of manifest's records
+        :return: List of entities passed the validation
         """
         validated_records = []
-        if not manifest_records:
-            raise EmptyManifestError
         for manifest_record in manifest_records:
-            manifest = manifest_record.get("entity")
+            manifest = manifest_record.entity
             if isinstance(manifest, dict) and manifest.get("kind"):
-                generic_schema = self.get_schema(manifest_record.get("schema"))
-                validation_result = self._validate_against_generic_schema(generic_schema, manifest) \
-                                    and self._validate_entity(manifest)
+                validation_result = self._validate_entity(manifest)
                 if validation_result:
                     validated_records.append(manifest_record)
             else:
                 raise NotOSDUSchemaFormatError(f"Not valid schema {manifest}")
         return validated_records
+
+    def ensure_manifest_validity(self, manifest: dict) -> dict:
+        """
+        Validate manifest entities inside manifest and return only valid entities with saved structure
+
+        """
+        if manifest.get("ReferenceData"):
+            for data in manifest["ReferenceData"]:
+                is_valid = self._validate_entity(data)
+                if not is_valid:
+                    logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                    manifest["ReferenceData"].remove(data)
+        if manifest.get("MasterData"):
+            master_data = manifest["MasterData"]
+            for data in master_data:
+                is_valid = self._validate_entity(data)
+                if not is_valid:
+                    logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                    manifest["MasterData"].remove(data)
+        if manifest.get("Data"):
+            if manifest["Data"].get("WorkProduct"):
+                work_product_data = manifest["Data"]["WorkProduct"]
+                is_valid = self._validate_entity(work_product_data)
+                if not is_valid:
+                    logger.warning(f"Resource with kind {work_product_data.get('kind')} was rejected")
+                    manifest["Data"]["WorkProduct"] = {}
+            if manifest["Data"].get("WorkProductComponents"):
+                wpc_data = manifest["Data"]["WorkProductComponents"]
+                for data in wpc_data:
+                    is_valid = self._validate_entity(data)
+                    if not is_valid:
+                        logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                        manifest["Data"]["WorkProductComponents"].remove(data)
+            if manifest["Data"].get("Datasets"):
+                datasets = manifest["Data"]["Datasets"]
+                for data in datasets:
+                    is_valid = self._validate_entity(data)
+                    if not is_valid:
+                        logger.warning(f"Resource with kind {data.get('kind')} was rejected")
+                        manifest["Data"]["Datasets"].remove(data)
+        return manifest
diff --git a/src/dags/osdu-ingest-r3.py b/src/dags/osdu-ingest-r3.py
index 1177305dc5fdd6860e243c77025595a9205ac7e6..286f7134beb2fb8172b4cc14e2f673e81ffa3690 100644
--- a/src/dags/osdu-ingest-r3.py
+++ b/src/dags/osdu-ingest-r3.py
@@ -19,8 +19,20 @@ from datetime import timedelta
 
 import airflow
 from airflow import DAG
+from airflow.models import Variable
+from airflow.operators.dummy_operator import DummyOperator
+from airflow.operators.python_operator import BranchPythonOperator
+from libs.exceptions import NotOSDUSchemaFormatError
+from operators.ensure_manifest_integrity import EnsureManifestIntegrityOperator
 from operators.process_manifest_r3 import ProcessManifestOperatorR3
 from operators.update_status import UpdateStatusOperator
+from operators.validate_manifest_schema import ValidateManifestSchemaOperator
+
+BATCH_NUMBER = int(Variable.get("core__ingestion__batch_count", "3"))
+PROCESS_SINGLE_MANIFEST_FILE = "process_single_manifest_file_task"
+PROCESS_BATCH_MANIFEST_FILE = "batch_upload"
+ENSURE_INTEGRITY_TASK = "provide_manifest_integrity_task"
+SINGLE_MANIFEST_FILE_FIRST_OPERATOR = "validate_manifest_schema_task"
 
 default_args = {
     "start_date": airflow.utils.dates.days_ago(0),
@@ -30,29 +42,90 @@ default_args = {
 }
 
 workflow_name = "Osdu_ingest"
-dag = DAG(
+
+
+def is_batch(**context):
+    """
+    :param context: Dag context
+    :return: SubDag to be executed next depending on Manifest type
+    """
+    manifest = context["dag_run"].conf["execution_context"].get("manifest")
+
+    if isinstance(manifest, dict):
+        subdag = SINGLE_MANIFEST_FILE_FIRST_OPERATOR
+    elif isinstance(manifest, list):
+        subdag = PROCESS_BATCH_MANIFEST_FILE
+    else:
+        raise NotOSDUSchemaFormatError(f"Manifest must be either 'dict' or 'list'. "
+                                       f"Got {manifest}.")
+    return subdag
+
+
+with DAG(
     workflow_name,
     default_args=default_args,
-    description="liveness monitoring dag",
+    description="R3 manifest processing with providing integrity",
     schedule_interval=None,
     dagrun_timeout=timedelta(minutes=60)
-)
-
-update_status_running_op = UpdateStatusOperator(
-    task_id="update_status_running_task",
-    dag=dag
-)
-
-update_status_finished_op = UpdateStatusOperator(
-    task_id="update_status_finished_task",
-    dag=dag,
-    trigger_rule="all_done",
-)
-
-process_manifest_op = ProcessManifestOperatorR3(
-    task_id="proccess_manifest_task",
-    provide_context=True,
-    dag=dag
-)
-
-update_status_running_op >> process_manifest_op >> update_status_finished_op  # pylint: disable=pointless-statement
+) as dag:
+    update_status_running_op = UpdateStatusOperator(
+        task_id="update_status_running_task",
+        dag=dag
+    )
+
+    branch_is_batch_op = BranchPythonOperator(
+        task_id="check_payload_type",
+        dag=dag,
+        provide_context=True,
+        python_callable=is_batch,
+        trigger_rule="none_failed_or_skipped"
+    )
+
+    update_status_finished_op = UpdateStatusOperator(
+        task_id="update_status_finished_task",
+        dag=dag,
+        trigger_rule="all_done",
+    )
+
+    validate_schema_operator = ValidateManifestSchemaOperator(
+        task_id="validate_manifest_schema_task",
+        provide_context=True,
+        dag=dag,
+        trigger_rule="none_failed_or_skipped"
+    )
+
+    ensure_integrity_op = EnsureManifestIntegrityOperator(
+        task_id=ENSURE_INTEGRITY_TASK,
+        provide_context=True,
+        previous_task_id=validate_schema_operator.task_id,
+        dag=dag,
+        trigger_rule="none_failed_or_skipped"
+    )
+
+    process_single_manifest_file = ProcessManifestOperatorR3(
+        task_id=PROCESS_SINGLE_MANIFEST_FILE,
+        provide_context=True,
+        dag=dag,
+        previous_task_id=ensure_integrity_op.task_id,
+        trigger_rule="none_failed_or_skipped"
+    )
+
+    # Dummy operator as entry point into parallel task of batch upload
+    batch_upload = DummyOperator(
+        dag=dag,
+        task_id=PROCESS_BATCH_MANIFEST_FILE
+    )
+
+    for batch in range(0, BATCH_NUMBER):
+        batch_upload >> ProcessManifestOperatorR3(
+            task_id=f"process_manifest_task_{batch + 1}",
+            provide_context=True,
+            dag=dag,
+            previous_task_id=f"provide_manifest_integrity_task_{batch + 1}",
+            batch_number=batch + 1,
+            trigger_rule="none_failed_or_skipped"
+        ) >> update_status_finished_op
+
+update_status_running_op >> branch_is_batch_op  # pylint: disable=pointless-statement
+branch_is_batch_op >> batch_upload  # pylint: disable=pointless-statement
+branch_is_batch_op >> validate_schema_operator >> ensure_integrity_op >> process_single_manifest_file >> update_status_finished_op  # pylint: disable=pointless-statement
diff --git a/src/dags/other-log-ingest.py b/src/dags/other-log-ingest.py
deleted file mode 100644
index 101a0dcf2e055a707a43b8f4844ed43b073622ef..0000000000000000000000000000000000000000
--- a/src/dags/other-log-ingest.py
+++ /dev/null
@@ -1,41 +0,0 @@
-#  Copyright 2020 Google LLC
-#
-#  Licensed under the Apache License, Version 2.0 (the "License");
-#  you may not use this file except in compliance with the License.
-#  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-
-"""Dummy DAG"""
-
-from datetime import timedelta
-
-import airflow
-from airflow import DAG
-from airflow.operators.bash_operator import BashOperator
-
-default_args = {
-    "start_date": airflow.utils.dates.days_ago(0),
-    "retries": 1,
-    "retry_delay": timedelta(minutes=50)
-}
-
-dag = DAG(
-    "Other_log_ingest",
-    default_args=default_args,
-    description="liveness monitoring dag",
-    schedule_interval=None,
-    dagrun_timeout=timedelta(minutes=60))
-
-t1 = BashOperator(
-    task_id="echo",
-    bash_command="echo test",
-    dag=dag,
-    depends_on_past=False,
-    priority_weight=2**31 - 1)
diff --git a/src/dags/providers/gcp/gcp_blob_storage_client.py b/src/dags/providers/gcp/gcp_blob_storage_client.py
index 10bd4b3f08efd936ede4f877b5a996fa261db534..8a6deae13979db012e77ba1647d6e4617653429e 100644
--- a/src/dags/providers/gcp/gcp_blob_storage_client.py
+++ b/src/dags/providers/gcp/gcp_blob_storage_client.py
@@ -104,7 +104,7 @@ class GoogleCloudStorageClient(BlobStorageClient):
         bucket = self._storage_client.bucket(bucket_name)
         blob = bucket.get_blob(source_blob_name)
 
-        file_as_bytes = blob.download_as_bytes()
+        file_as_bytes = blob.download_as_string()
         logger.debug(f"File {source_blob_name} got from bucket {bucket_name}.")
 
         return file_as_bytes, blob.content_type
diff --git a/src/dags/well-log-ingest.py b/src/dags/well-log-ingest.py
deleted file mode 100644
index bca78dd7ddd1dbad57d63d84bf7844eab99db845..0000000000000000000000000000000000000000
--- a/src/dags/well-log-ingest.py
+++ /dev/null
@@ -1,41 +0,0 @@
-#  Copyright 2020 Google LLC
-#
-#  Licensed under the Apache License, Version 2.0 (the "License");
-#  you may not use this file except in compliance with the License.
-#  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-
-"""WellLog DAG."""
-
-from datetime import timedelta
-
-import airflow
-from airflow import DAG
-from airflow.operators.bash_operator import BashOperator
-
-default_args = {
-    "start_date": airflow.utils.dates.days_ago(0),
-    "retries": 1,
-    "retry_delay": timedelta(minutes=50)
-}
-
-dag = DAG(
-    "Well_log_ingest",
-    default_args=default_args,
-    description="liveness monitoring dag",
-    schedule_interval=None,
-    dagrun_timeout=timedelta(minutes=60))
-
-t1 = BashOperator(
-    task_id="echo",
-    bash_command="echo test",
-    dag=dag,
-    depends_on_past=False,
-    priority_weight=2**31 - 1)
diff --git a/src/plugins/operators/deprecated/update_status.py b/src/plugins/operators/deprecated/update_status.py
index dfb219740bd502d4c84e1527d16c1859c0a5feae..8c58d6b34eb9c823b616ff3b35d6ccf8ab1c47f3 100644
--- a/src/plugins/operators/deprecated/update_status.py
+++ b/src/plugins/operators/deprecated/update_status.py
@@ -84,7 +84,7 @@ class UpdateStatusOperator(BaseOperator):
         status = self.status.value
         status_updater = UpdateStatus(
             workflow_name="",
-            workflow_url=Variable.get("update_status_url"),
+            workflow_url=Variable.get("core__service__workflow__url"),
             workflow_id=workflow_id,
             run_id="",
             status=status,
diff --git a/src/plugins/operators/ensure_manifest_integrity.py b/src/plugins/operators/ensure_manifest_integrity.py
new file mode 100644
index 0000000000000000000000000000000000000000..01d5d34b0eebdb164ccc15840d56fc54aa4ac536
--- /dev/null
+++ b/src/plugins/operators/ensure_manifest_integrity.py
@@ -0,0 +1,66 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+"""R3 Validate reference Manifest operator."""
+import logging
+from typing import List, TypeVar
+
+from airflow.utils import apply_defaults
+from airflow.models import BaseOperator, Variable
+from libs.context import Context
+from libs.validation.validate_referential_integrity import ManifestIntegrity
+from libs.refresh_token import AirflowTokenRefresher
+from operators.mixins.ReceivingContextMixin import ReceivingContextMixin
+
+logger = logging.getLogger()
+
+
+class EnsureManifestIntegrityOperator(BaseOperator, ReceivingContextMixin):
+    """Operator to validate ref inside manifest R3 and remove invalid entities."""
+
+    ui_color = '#dad5ff'
+    ui_fgcolor = '#000000'
+
+    @apply_defaults
+    def __init__(self,
+                 previous_task_id: str=None,
+                 *args, **kwargs):
+        """Init base operator and obtain base urls from Airflow Variables."""
+        super().__init__(*args, **kwargs)
+        self.search_url = Variable.get('core__service__search__url')
+        self.previous_task_id = previous_task_id
+
+    def execute(self, context: dict):
+        """Execute manifest validation then process it.
+
+        :param context: Airflow context
+        :type context: dict
+        """
+        payload_context = Context.populate(context["dag_run"].conf["execution_context"])
+        token_refresher = AirflowTokenRefresher()
+
+        manifest_integrity = ManifestIntegrity(
+            self.search_url,
+            token_refresher,
+            payload_context,
+        )
+
+        execution_context = context["dag_run"].conf["execution_context"]
+        manifest_data = self._get_manifest_data(context, execution_context)
+        logger.debug(f"Manifest data: {manifest_data}")
+
+        manifest_integrity.ensure_integrity(manifest_data)
+        logger.debug(f"Valid manifest data: {manifest_data}")
+        return {"manifest": manifest_data}
diff --git a/src/plugins/operators/mixins/ReceivingContextMixin.py b/src/plugins/operators/mixins/ReceivingContextMixin.py
new file mode 100644
index 0000000000000000000000000000000000000000..3cba452426d51f55153a254416f8790951b58aad
--- /dev/null
+++ b/src/plugins/operators/mixins/ReceivingContextMixin.py
@@ -0,0 +1,36 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+from libs.types import ManifestType
+
+
+class ReceivingContextMixin:
+    """Mixin for receiving manifest file from XCOMs in case if current operator not the first in the row"""
+
+    def _get_manifest_data(self, context: dict, execution_context: dict) -> ManifestType:
+        """
+        Receive manifest file. If previous task id not None - get manifest file from XCOMs.
+        Otherwise - get manifest file from execution context
+        """
+        if self.previous_task_id:
+            previous_task_value = context["ti"].xcom_pull(task_ids=self.previous_task_id,
+                                                          key="return_value")
+            if previous_task_value:
+                manifest_data = previous_task_value["manifest"]
+            else:
+                manifest_data = execution_context["manifest"]
+        else:
+            manifest_data = execution_context["manifest"]
+        return manifest_data
diff --git a/tests/end-to-end-tests/mock-data/default-ingest-valid.json b/src/plugins/operators/mixins/__init__.py
similarity index 57%
rename from tests/end-to-end-tests/mock-data/default-ingest-valid.json
rename to src/plugins/operators/mixins/__init__.py
index 0dc45d4a7e7d6f3a5ec556b427d77cc4669c37cb..de8f5ce5bf56bdd78824065dbbd52846984960b1 100644
--- a/tests/end-to-end-tests/mock-data/default-ingest-valid.json
+++ b/src/plugins/operators/mixins/__init__.py
@@ -1,5 +1,5 @@
-#  Copyright 2020 Google LLC
-#  Copyright 2020 EPAM Systems
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
 #
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.
@@ -12,12 +12,3 @@
 #  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
-
-{
-    "authorization": "Bearer foo",
-    "legal-tags": "\"legaltags\": [\"foo\"],\"otherRelevantDataCountries\": [\"FR\",\"US\",\"CA\"]\", \"data\": {\"FileID\": \"83d0ec02be114cf580da81f96b2a831e\"}",
-    "AppKey": "test",
-    "acl": "\"viewers\": [\"foo\"], \"owners\": [\"foo\"]",
-    "data-partition-id": "odes",
-    "WorkflowID": "foo"
-}
\ No newline at end of file
diff --git a/src/plugins/operators/process_manifest_r2.py b/src/plugins/operators/process_manifest_r2.py
index e85ad3aeea2f41da20b7eaac740fef064a3ad33c..85640d3428c610adb8822ca2a30d112db9f3e9f0 100644
--- a/src/plugins/operators/process_manifest_r2.py
+++ b/src/plugins/operators/process_manifest_r2.py
@@ -30,11 +30,11 @@ from urllib.error import HTTPError
 import requests
 import tenacity
 from airflow.models import BaseOperator, Variable
-from libs.refresh_token import AirflowTokenRefresher, refresh_token
-
+from libs.refresh_token import AirflowTokenRefresher
+from osdu_api.libs.auth.authorization import authorize
 
 config = configparser.RawConfigParser()
-config.read(Variable.get("dataload_config_path"))
+config.read(Variable.get("core__config__dataload_config_path"))
 
 DEFAULT_TENANT = config.get("DEFAULTS", "tenant")
 DEFAULT_SOURCE = config.get("DEFAULTS", "authority")
@@ -306,7 +306,7 @@ def create_workproduct_request_data(loaded_conf: dict, product_type: str, wp, wp
     stop=tenacity.stop_after_attempt(RETRIES),
     reraise=True
 )
-@refresh_token(AirflowTokenRefresher())
+@authorize(AirflowTokenRefresher())
 def send_request(headers, request_data):
     """Send request to records storage API."""
 
@@ -317,7 +317,7 @@ def send_request(headers, request_data):
     for retry in range(retries):
         try:
             # send batch request for creating records
-            response = requests.put(Variable.get('storage_url'), json.dumps(request_data),
+            response = requests.put(Variable.get('core__service__storage__url'), json.dumps(request_data),
                                     headers=headers)
 
             if response.status_code in DATA_LOAD_OK_RESPONSE_CODES:
diff --git a/src/plugins/operators/process_manifest_r3.py b/src/plugins/operators/process_manifest_r3.py
index 37218db447d498f800e987066c7af0e5785bbf1f..5c3744c590a9f26db38ed30651cacf0f03c3997e 100644
--- a/src/plugins/operators/process_manifest_r3.py
+++ b/src/plugins/operators/process_manifest_r3.py
@@ -13,74 +13,157 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-"""R2 Process Manifest operator."""
+
+"""
+R3 Process Manifest operator.
+"""
 
 import logging
+from math import ceil
+from typing import List, Tuple
+
 from airflow.utils import apply_defaults
 from airflow.models import BaseOperator, Variable
+from jsonschema import SchemaError
 from libs.context import Context
+from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError, \
+    UploadFileError, GetSchemaError
 from libs.source_file_check import SourceFileChecker
 from libs.handle_file import FileHandler
 from libs.refresh_token import AirflowTokenRefresher
 from libs.process_manifest_r3 import ManifestProcessor
-from libs.traverse_manifest import ManifestTraversal
-from libs.validate_schema import SchemaValidator
+from libs.processors.single_manifest_processor import SingleManifestProcessor
+from libs.search_client import SearchClient
+from libs.types import ManifestType
+from libs.validation.validate_data_integrity import DataIntegrityValidator
+from libs.validation.validate_file_source import FileSourceValidator
+from libs.validation.validate_referential_integrity import ManifestIntegrity
+from libs.validation.validate_schema import SchemaValidator
+from requests import HTTPError
+from operators.mixins.ReceivingContextMixin import ReceivingContextMixin
 
 logger = logging.getLogger()
 
 
-class ProcessManifestOperatorR3(BaseOperator):
+class ProcessManifestOperatorR3(BaseOperator, ReceivingContextMixin):
     """Operator to process manifest R3."""
 
     ui_color = '#dad5ff'
     ui_fgcolor = '#000000'
 
     @apply_defaults
-    def __init__(self, *args, **kwargs):
+    def __init__(self, previous_task_id: str = None, batch_number=3, *args, **kwargs):
         """Init base operator and obtain base urls from Airflow Variables."""
         super().__init__(*args, **kwargs)
-        self.schema_service_url = Variable.get('schema_service_url')
-        self.storage_url = Variable.get('storage_url')
-        self.file_service_url = Variable.get('file_service_url')
+        self.previous_task_id = previous_task_id
+        self.batch_number = batch_number
+        self.schema_service_url = Variable.get('core__service__schema__url')
+        self.search_service_url = Variable.get('core__service__search__url')
+        self.storage_url = Variable.get('core__service__storage__url')
+        self.file_service_host = Variable.get('core__service__file__host')
+        self.batch_count = int(Variable.get("core__ingestion__batch_count", "3"))
+
+    def _get_manifest_files_range(self, manifests: List[dict]) -> Tuple[int, int]:
+        """
+        Get start and end indexes of a manifest files slice to be processed within this task.
+        :param manifests: A list of manifests
+        :return: start index, end index
+        """
+        split_size = ceil(len(manifests) / self.batch_count)
+        slice_start_index = (self.batch_number - 1) * split_size
+        slice_end_index = self.batch_number * split_size
+        return slice_start_index, slice_end_index
+
+    def _process_manifest(
+        self,
+        single_manifest_processor: SingleManifestProcessor,
+        manifest: ManifestType
+    ) -> List[str]:
+        """
+
+        :param single_manifest_processor: Object to process a single manifest file.
+        Processing includes validation against schemas, storing records enc.
+        :param manifest: A single manifest file or a list of them.
+        :return:
+        """
+        if isinstance(manifest, dict):
+            record_ids = single_manifest_processor.process_manifest(manifest, False)
+
+        elif isinstance(manifest, list):
+            record_ids = []
+            slice_start_index, slice_end_index = self._get_manifest_files_range(manifest)
+            logger.debug(f"Start and indexes {slice_start_index}:{slice_end_index}")
+            for single_manifest in manifest[slice_start_index:slice_end_index]:
+                logger.debug(f"processing {single_manifest}")
+                try:
+                    record_ids.extend(single_manifest_processor.process_manifest(single_manifest, True))
+                except (UploadFileError, HTTPError, GetSchemaError, SchemaError) as e:
+                    logger.error(f"Can't process {single_manifest}")
+                    logger.error(e)
+                    continue
+        else:
+            raise NotOSDUSchemaFormatError(
+                f"Manifest {manifest} must be either not empty 'list' or 'dict'")
+
+        return record_ids
 
     def execute(self, context: dict):
         """Execute manifest validation then process it.
 
-        Execution steps:
-        1) initialize schema validator
-        2) validate manifest file against common schema
-        3) traverse manifest file and extract manifest entities
-        4) validate extracted manifest entities
-        5) process valid manifest entities
+        Get a single manifest file or a list of them.
+        If it is a list, calculate which range (slice) of manifest files must be processed and then
+        process this range one by one.
 
         :param context: Airflow context
         :type context: dict
         """
-
         execution_context = context["dag_run"].conf["execution_context"]
         payload_context = Context.populate(execution_context)
         token_refresher = AirflowTokenRefresher()
-        file_handler = FileHandler(self.file_service_url, token_refresher, payload_context)
+        file_handler = FileHandler(self.file_service_host, token_refresher, payload_context)
+        file_source_validator = FileSourceValidator()
+        search_client = SearchClient(self.search_service_url, token_refresher, payload_context)
         source_file_checker = SourceFileChecker()
 
-        validator = SchemaValidator(
-            self.schema_service_url,
+        data_integrity_validator = DataIntegrityValidator(
+            search_client=search_client,
+            file_source_validator=file_source_validator
+        )
+
+        referential_integrity_validator = ManifestIntegrity(
+            self.search_service_url,
             token_refresher,
             payload_context
         )
-        manifest_schema = validator.validate_common_schema(execution_context)
-        traversal = ManifestTraversal(execution_context, manifest_schema)
-        manifest_entities = traversal.traverse_manifest()
-        logger.debug(f"entities count: {len(manifest_entities)}")
-        valid_manifest_entities = validator.validate_manifest(manifest_entities)
-        logger.debug(f"valid entities count: {len(valid_manifest_entities)}")
+
         manifest_processor = ManifestProcessor(
-            self.storage_url,
-            valid_manifest_entities,
-            file_handler,
-            source_file_checker,
+            storage_url=self.storage_url,
+            file_handler=file_handler,
+            token_refresher=token_refresher,
+            context=payload_context,
+            source_file_checker=source_file_checker,
+        )
+        validator = SchemaValidator(
+            self.schema_service_url,
             token_refresher,
-            payload_context,
+            payload_context
         )
-        record_ids = manifest_processor.process_manifest()
+        single_manifest_processor = SingleManifestProcessor(
+            storage_url=self.storage_url,
+            payload_context=payload_context,
+            data_integrity_validator=data_integrity_validator,
+            referential_integrity_validator=referential_integrity_validator,
+            manifest_processor=manifest_processor,
+            schema_validator=validator,
+            token_refresher=token_refresher,
+        )
+
+        manifest_data = self._get_manifest_data(context, execution_context)
+        logger.debug(f"Manifest data: {manifest_data}")
+
+        if not manifest_data:
+            raise EmptyManifestError(
+                f"Data {context['dag_run'].conf} doesn't contain 'manifest field'")
+        record_ids = self._process_manifest(single_manifest_processor, manifest_data)
+        logger.info(f"Processed ids {record_ids}")
         context["ti"].xcom_push(key="record_ids", value=record_ids)
diff --git a/src/plugins/operators/search_record_id.py b/src/plugins/operators/search_record_id.py
index d457c252258b428747ba499ac385b0f47406ec79..2693f15b0048f5a92ddfe7ce411311a5c4c0bca0 100644
--- a/src/plugins/operators/search_record_id.py
+++ b/src/plugins/operators/search_record_id.py
@@ -45,6 +45,6 @@ class SearchRecordIdOperator(BaseOperator):
         """
         payload_context = Context.populate(context["dag_run"].conf)
         record_ids = context["ti"].xcom_pull(key="record_ids", )
-        ids_searcher = SearchId(Variable.get("search_url"), record_ids, AirflowTokenRefresher(),
+        ids_searcher = SearchId(Variable.get("core__service__search__url"), record_ids, AirflowTokenRefresher(),
                                 payload_context)
         ids_searcher.check_records_searchable()
diff --git a/src/plugins/operators/update_status.py b/src/plugins/operators/update_status.py
index 3434c9622b46a058c79b7099eb71e70e09bc55a3..3b2a5a20b84729d73305897536e91537c8eae912 100644
--- a/src/plugins/operators/update_status.py
+++ b/src/plugins/operators/update_status.py
@@ -86,7 +86,7 @@ class UpdateStatusOperator(BaseOperator):
         status = self.status.value
         status_updater = UpdateStatus(
             workflow_name=workflow_name,
-            workflow_url=Variable.get("workflow_url"),
+            workflow_url=Variable.get("core__service__workflow__host"),
             workflow_id="",
             run_id=run_id,
             status=status,
diff --git a/src/plugins/operators/validate_manifest_schema.py b/src/plugins/operators/validate_manifest_schema.py
new file mode 100644
index 0000000000000000000000000000000000000000..cdbd519730c9fe2f2650028fb01c09227358c400
--- /dev/null
+++ b/src/plugins/operators/validate_manifest_schema.py
@@ -0,0 +1,75 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+"""
+Validate Manifest against R3 schemas operator.
+"""
+
+import logging
+
+from airflow.utils import apply_defaults
+from airflow.models import BaseOperator, Variable
+from libs.context import Context
+from libs.exceptions import EmptyManifestError
+from libs.refresh_token import AirflowTokenRefresher
+from libs.validation.validate_schema import SchemaValidator
+from operators.mixins.ReceivingContextMixin import ReceivingContextMixin
+
+logger = logging.getLogger()
+
+
+class ValidateManifestSchemaOperator(BaseOperator, ReceivingContextMixin):
+    """Operator to validate manifest against definition schemasR3."""
+
+    ui_color = '#dad5ff'
+    ui_fgcolor = '#000000'
+
+    @apply_defaults
+    def __init__(self, previous_task_id: str = None, *args, **kwargs):
+        """Init base operator and obtain base urls from Airflow Variables."""
+        super().__init__(*args, **kwargs)
+        self.previous_task_id = previous_task_id
+        self.schema_service_url = Variable.get('core__service__schema__url')
+
+    def execute(self, context: dict):
+        """Execute manifest validation then process it.
+
+        Get a single manifest file or a list of them.
+        If it is a list, calculate which range (slice) of manifest files must be processed and then
+        process this range one by one.
+
+        :param context: Airflow context
+        :type context: dict
+        """
+        execution_context = context["dag_run"].conf["execution_context"]
+        payload_context = Context.populate(execution_context)
+        token_refresher = AirflowTokenRefresher()
+
+        schema_validator = SchemaValidator(
+            self.schema_service_url,
+            token_refresher,
+            payload_context
+        )
+
+        manifest_data = self._get_manifest_data(context, execution_context)
+        logger.debug(f"Manifest data: {manifest_data}")
+
+        if not manifest_data:
+            raise EmptyManifestError(
+                f"Data {context['dag_run'].conf} doesn't contain 'manifest field'")
+
+        _ = schema_validator.validate_common_schema(manifest_data)
+        valid_manifest_file = schema_validator.ensure_manifest_validity(manifest_data)
+        return {"manifest": valid_manifest_file}
diff --git a/tests/plugin-unit-tests/data/data_integrity/empty_data.json b/tests/plugin-unit-tests/data/data_integrity/empty_data.json
new file mode 100644
index 0000000000000000000000000000000000000000..50e3272cc2d607777608a9b2179ef783806e5a74
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/empty_data.json
@@ -0,0 +1,6 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {}
+}
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/empty_data_inside.json b/tests/plugin-unit-tests/data/data_integrity/empty_data_inside.json
new file mode 100644
index 0000000000000000000000000000000000000000..aa3c17bcbaf2b2aac453123234eea4d46cf259b5
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/empty_data_inside.json
@@ -0,0 +1,9 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProductComponents": [],
+        "Datasets": []
+    }
+}
diff --git a/tests/plugin-unit-tests/data/data_integrity/empty_wp.json b/tests/plugin-unit-tests/data/data_integrity/empty_wp.json
new file mode 100644
index 0000000000000000000000000000000000000000..4e62d45bd2c1706a7c3928673d662ab14b6eb331
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/empty_wp.json
@@ -0,0 +1,73 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {},
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-1"
+                    ]
+                }
+            },
+            {
+                "id": "surrogate-key:wpc-2",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-2"
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
diff --git a/tests/plugin-unit-tests/data/data_integrity/file_source/invalid_file_collections.json b/tests/plugin-unit-tests/data/data_integrity/file_source/invalid_file_collections.json
new file mode 100644
index 0000000000000000000000000000000000000000..905b20b85200322f13ec92a061a82c654c023367
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/file_source/invalid_file_collections.json
@@ -0,0 +1,63 @@
+[
+    {
+        "id": "surrogate-key:file-1",
+        "kind": "osdu:wks:dataset--FileCollection.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfos": [
+                    {
+                        "FileSource": "",
+                        "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                    },
+                    {
+                        "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_12.pdf",
+                        "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_12.pdf"
+                    }
+                ]
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:file-2",
+        "kind": "osdu:wks:dataset--FileCollection.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfos": []
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:file-3",
+        "kind": "osdu:wks:dataset--FileCollection.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {}
+        }
+    }
+]
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/file_source/invalid_files.json b/tests/plugin-unit-tests/data/data_integrity/file_source/invalid_files.json
new file mode 100644
index 0000000000000000000000000000000000000000..078613db8349af8f158cc90ffcaab6834f181870
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/file_source/invalid_files.json
@@ -0,0 +1,57 @@
+[
+    {
+        "id": "surrogate-key:file-1",
+        "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfo": {
+                    "FileSource": "",
+                    "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                }
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:file-2",
+        "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfo": {}
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:file-3",
+        "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {}
+        }
+    }
+]
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/file_source/valid_file_collections.json b/tests/plugin-unit-tests/data/data_integrity/file_source/valid_file_collections.json
new file mode 100644
index 0000000000000000000000000000000000000000..c4fd7fdf033cb5a1f03d32fe7a0d1dfc1470bcdc
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/file_source/valid_file_collections.json
@@ -0,0 +1,47 @@
+[
+    {
+        "id": "surrogate-key:file-1",
+        "kind": "osdu:wks:dataset--FileCollection.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "IndexFilePath": "someIndexFilePathString"
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:file-2",
+        "kind": "osdu:wks:dataset--FileCollection.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfos": [
+                    {
+                        "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                        "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                    },
+                    {
+                        "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_12.pdf",
+                        "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_12.pdf"
+                    }
+                ]
+            }
+        }
+    }
+]
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/file_source/valid_files.json b/tests/plugin-unit-tests/data/data_integrity/file_source/valid_files.json
new file mode 100644
index 0000000000000000000000000000000000000000..4bbaeca492edc977d1e4d536e50d3aca0a7621b1
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/file_source/valid_files.json
@@ -0,0 +1,43 @@
+[
+    {
+        "id": "surrogate-key:file-1",
+        "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfo": {
+                    "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                    "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                }
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:file-2",
+        "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "DatasetProperties": {
+                "FileSourceInfo": {
+                    "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                }
+            }
+        }
+    }
+]
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/invalid_wp.json b/tests/plugin-unit-tests/data/data_integrity/invalid_wp.json
new file mode 100644
index 0000000000000000000000000000000000000000..bf963009fc2f73f007cd1c404b362ae68a9ca83c
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/invalid_wp.json
@@ -0,0 +1,92 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "69_D_CH_11",
+                "Description": "Document",
+                "Components": [
+                    "surrogate-key:wpc-1",
+                    "surrogate-key:wpc-2"
+                ]
+            }
+        },
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-1"
+                    ]
+                }
+            },
+            {
+                "id": "surrogate-key:wpc-2",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-2"
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/orphan_datasets.json b/tests/plugin-unit-tests/data/data_integrity/orphan_datasets.json
new file mode 100644
index 0000000000000000000000000000000000000000..895b357f4cc56543c06921e70d73e58b2ea55e54
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/orphan_datasets.json
@@ -0,0 +1,92 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "69_D_CH_11",
+                "Description": "Document",
+                "Components": [
+                    "surrogate-key:wpc-1"
+                ]
+            }
+        },
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-1"
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            },
+            {
+                "id": "surrogate-key:file-2",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/valid_data.json b/tests/plugin-unit-tests/data/data_integrity/valid_data.json
new file mode 100644
index 0000000000000000000000000000000000000000..24bfbf216cf832e90217f4ea3d4ea66d6deb159d
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/valid_data.json
@@ -0,0 +1,71 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "69_D_CH_11",
+                "Description": "Document",
+                "Components": [
+                    "surrogate-key:wpc-1"
+                ]
+            }
+        },
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-1"
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/valid_data_real_ids.json b/tests/plugin-unit-tests/data/data_integrity/valid_data_real_ids.json
new file mode 100644
index 0000000000000000000000000000000000000000..83a02cfddda120fd69e178b909207c3136aa7c49
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/valid_data_real_ids.json
@@ -0,0 +1,73 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "69_D_CH_11",
+                "Description": "Document",
+                "Components": [
+                    "surrogate-key:wpc-1",
+                    "opendes:work-product-component--GenericWorkProductComponent:1234"
+                ]
+            }
+        },
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-1",
+                        "opendes:dataset--GenericDataset:1234"
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/data_integrity/valid_wp_invalid_wpc.json b/tests/plugin-unit-tests/data/data_integrity/valid_wp_invalid_wpc.json
new file mode 100644
index 0000000000000000000000000000000000000000..4c400dbd61ab91d1929c5f15b38af29971aa4c10
--- /dev/null
+++ b/tests/plugin-unit-tests/data/data_integrity/valid_wp_invalid_wpc.json
@@ -0,0 +1,91 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "69_D_CH_11",
+                "Description": "Document",
+                "Components": [
+                    "surrogate-key:wpc-1"
+                ]
+            }
+        },
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-1"
+                    ]
+                }
+            },
+            {
+                "id": "surrogate-key:wpc-2",
+                "kind": "osdu:wks:work-product-component--Document:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "69_D_CH_11",
+                    "Description": "Document",
+                    "Datasets": [
+                        "surrogate-key:file-2"
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "/r1/data/provided/USGS_docs/69_D_CH_11.pdf",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/USGS_docs/69_D_CH_11.pdf"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
\ No newline at end of file
diff --git a/tests/plugin-unit-tests/data/manifests/new_schema_Manifest.1.0.0.json b/tests/plugin-unit-tests/data/manifests/new_schema_Manifest.1.0.0.json
new file mode 100644
index 0000000000000000000000000000000000000000..2ccfa2e9ab4230361c69760731398d39f1a2f148
--- /dev/null
+++ b/tests/plugin-unit-tests/data/manifests/new_schema_Manifest.1.0.0.json
@@ -0,0 +1,2403 @@
+{
+    "x-osdu-inheriting-from-kind": [],
+    "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+    "$schema": "http://json-schema.org/draft-07/schema#",
+    "x-osdu-schema-source": "osdu:wks:Manifest:1.0.0",
+    "description": "Load manifest applicable for all types defined as 'kind', i.e. registered as schemas with the Schema Service. It supports loading of individual 'records' of any group-type or combinations. The load sequence follows a well-defined sequence. The 'ReferenceData' array is processed first (if populated). The 'MasterData' array is processed second (if populated) second. The 'Data' structure is processed last (if populated). Inside the 'Data' property the 'Datasets' array is processed first, followed by the 'WorkProductComponents' array, the 'WorkProduct' is processed last. Any arrays are ordered. should there be interdependencies, the dependent items must be placed behind their relationship targets, e.g. a master-data Well record must placed in the 'MasterData' array before its Wellbores.",
+    "title": "Load Manifest Schema",
+    "type": "object",
+    "definitions": {
+        "osdu:wks:AbstractCommonResources:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractCommonResources:1.0.0",
+            "description": "Common resources to be injected at root 'data' level for every entity, which is persistable in Storage. The insertion is performed by the OsduSchemaComposer script.",
+            "title": "OSDU Common Resources",
+            "type": "object",
+            "properties": {
+                "ResourceHomeRegionID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-OSDURegion:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "The name of the home [cloud environment] region for this OSDU resource object.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "OSDURegion",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Resource Home Region ID",
+                    "type": "string"
+                },
+                "ResourceHostRegionIDs": {
+                    "description": "The name of the host [cloud environment] region(s) for this OSDU resource object.",
+                    "title": "Resource Host Region ID",
+                    "type": "array",
+                    "items": {
+                        "x-osdu-relationship": [
+                            {
+                                "EntityType": "OSDURegion",
+                                "GroupType": "reference-data"
+                            }
+                        ],
+                        "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-OSDURegion:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                        "type": "string"
+                    }
+                },
+                "ResourceLifecycleStatus": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ResourceLifecycleStatus:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "Describes the current Resource Lifecycle status.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "ResourceLifecycleStatus",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Resource Lifecycle Status",
+                    "type": "string"
+                },
+                "ResourceSecurityClassification": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ResourceSecurityClassification:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "Classifies the security level of the resource.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "ResourceSecurityClassification",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Resource Security Classification",
+                    "type": "string"
+                },
+                "ResourceCurationStatus": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ResourceCurationStatus:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "Describes the current Curation status.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "ResourceCurationStatus",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Resource Curation Status",
+                    "type": "string"
+                },
+                "ExistenceKind": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ExistenceKind:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "Where does this data resource sit in the cradle-to-grave span of its existence?",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "ExistenceKind",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Existence Kind",
+                    "type": "string"
+                },
+                "Source": {
+                    "title": "Data Source",
+                    "type": "string",
+                    "description": "The entity that produced the record, or from which it is received; could be an organization, agency, system, internal team, or individual. For informational purposes only, the list of sources is not governed."
+                }
+            },
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractCommonResources.1.0.0.json"
+        },
+        "osdu:wks:AbstractMetaItem:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "oneOf": [
+                {
+                    "title": "FrameOfReferenceUOM",
+                    "type": "object",
+                    "properties": {
+                        "name": {
+                            "description": "The unit symbol or name of the unit.",
+                            "title": "UOM Unit Symbol",
+                            "type": "string",
+                            "example": "ft[US]"
+                        },
+                        "persistableReference": {
+                            "description": "The self-contained, persistable reference string uniquely identifying the Unit.",
+                            "title": "UOM Persistable Reference",
+                            "type": "string",
+                            "example": "{\"abcd\":{\"a\":0.0,\"b\":1200.0,\"c\":3937.0,\"d\":0.0},\"symbol\":\"ft[US]\",\"baseMeasurement\":{\"ancestry\":\"L\",\"type\":\"UM\"},\"type\":\"UAD\"}"
+                        },
+                        "unitOfMeasureID": {
+                            "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-UnitOfMeasure:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                            "description": "SRN to unit of measure reference.",
+                            "x-osdu-relationship": [
+                                {
+                                    "EntityType": "UnitOfMeasure",
+                                    "GroupType": "reference-data"
+                                }
+                            ],
+                            "type": "string",
+                            "example": "namespace:reference-data--UnitOfMeasure:ftUS:"
+                        },
+                        "kind": {
+                            "const": "Unit",
+                            "title": "UOM Reference Kind",
+                            "description": "The kind of reference, 'Unit' for FrameOfReferenceUOM."
+                        },
+                        "propertyNames": {
+                            "description": "The list of property names, to which this meta data item provides Unit context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"Data.StructureA.PropertyB\" to define a unique context.",
+                            "title": "UOM Property Names",
+                            "type": "array",
+                            "items": {
+                                "type": "string"
+                            },
+                            "example": [
+                                "HorizontalDeflection.EastWest",
+                                "HorizontalDeflection.NorthSouth"
+                            ]
+                        }
+                    },
+                    "required": [
+                        "kind",
+                        "persistableReference"
+                    ]
+                },
+                {
+                    "title": "FrameOfReferenceCRS",
+                    "type": "object",
+                    "properties": {
+                        "name": {
+                            "description": "The name of the CRS.",
+                            "title": "CRS Name",
+                            "type": "string",
+                            "example": "NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]"
+                        },
+                        "coordinateReferenceSystemID": {
+                            "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-CoordinateReferenceSystem:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                            "description": "SRN to CRS reference.",
+                            "x-osdu-relationship": [
+                                {
+                                    "EntityType": "CoordinateReferenceSystem",
+                                    "GroupType": "reference-data"
+                                }
+                            ],
+                            "type": "string",
+                            "example": "namespace:reference-data--CoordinateReferenceSystem:EPSG.32615:"
+                        },
+                        "persistableReference": {
+                            "description": "The self-contained, persistable reference string uniquely identifying the CRS.",
+                            "title": "CRS Persistable Reference",
+                            "type": "string",
+                            "example": "{\"authCode\":{\"auth\":\"EPSG\",\"code\":\"32615\"},\"type\":\"LBC\",\"ver\":\"PE_10_3_1\",\"name\":\"WGS_1984_UTM_Zone_15N\",\"wkt\":\"PROJCS[\\\"WGS_1984_UTM_Zone_15N\\\",GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Transverse_Mercator\\\"],PARAMETER[\\\"False_Easting\\\",500000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",-93.0],PARAMETER[\\\"Scale_Factor\\\",0.9996],PARAMETER[\\\"Latitude_Of_Origin\\\",0.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",32615]]\"}"
+                        },
+                        "kind": {
+                            "const": "CRS",
+                            "title": "CRS Reference Kind",
+                            "description": "The kind of reference, constant 'CRS' for FrameOfReferenceCRS."
+                        },
+                        "propertyNames": {
+                            "description": "The list of property names, to which this meta data item provides CRS context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"Data.StructureA.PropertyB\" to define a unique context.",
+                            "title": "CRS Property Names",
+                            "type": "array",
+                            "items": {
+                                "type": "string"
+                            },
+                            "example": [
+                                "KickOffPosition.X",
+                                "KickOffPosition.Y"
+                            ]
+                        }
+                    },
+                    "required": [
+                        "kind",
+                        "persistableReference"
+                    ]
+                },
+                {
+                    "title": "FrameOfReferenceDateTime",
+                    "type": "object",
+                    "properties": {
+                        "name": {
+                            "description": "The name of the DateTime format and reference.",
+                            "title": "DateTime Name",
+                            "type": "string",
+                            "example": "UTC"
+                        },
+                        "persistableReference": {
+                            "description": "The self-contained, persistable reference string uniquely identifying DateTime reference.",
+                            "title": "DateTime Persistable Reference",
+                            "type": "string",
+                            "example": "{\"format\":\"yyyy-MM-ddTHH:mm:ssZ\",\"timeZone\":\"UTC\",\"type\":\"DTM\"}"
+                        },
+                        "kind": {
+                            "const": "DateTime",
+                            "title": "DateTime Reference Kind",
+                            "description": "The kind of reference, constant 'DateTime', for FrameOfReferenceDateTime."
+                        },
+                        "propertyNames": {
+                            "description": "The list of property names, to which this meta data item provides DateTime context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"Data.StructureA.PropertyB\" to define a unique context.",
+                            "title": "DateTime Property Names",
+                            "type": "array",
+                            "items": {
+                                "type": "string"
+                            },
+                            "example": [
+                                "Acquisition.StartTime",
+                                "Acquisition.EndTime"
+                            ]
+                        }
+                    },
+                    "required": [
+                        "kind",
+                        "persistableReference"
+                    ]
+                },
+                {
+                    "title": "FrameOfReferenceAzimuthReference",
+                    "type": "object",
+                    "properties": {
+                        "name": {
+                            "description": "The name of the CRS or the symbol/name of the unit.",
+                            "title": "AzimuthReference Name",
+                            "type": "string",
+                            "example": "TrueNorth"
+                        },
+                        "persistableReference": {
+                            "description": "The self-contained, persistable reference string uniquely identifying AzimuthReference.",
+                            "title": "AzimuthReference Persistable Reference",
+                            "type": "string",
+                            "example": "{\"code\":\"TrueNorth\",\"type\":\"AZR\"}"
+                        },
+                        "kind": {
+                            "const": "AzimuthReference",
+                            "title": "AzimuthReference Reference Kind",
+                            "description": "The kind of reference, constant 'AzimuthReference', for FrameOfReferenceAzimuthReference."
+                        },
+                        "propertyNames": {
+                            "description": "The list of property names, to which this meta data item provides AzimuthReference context to. Data structures, which come in a single frame of reference, can register the property name, others require a full path like \"Data.StructureA.PropertyB\" to define a unique context.",
+                            "title": "AzimuthReference Property Names",
+                            "type": "array",
+                            "items": {
+                                "type": "string"
+                            },
+                            "example": [
+                                "Bearing"
+                            ]
+                        }
+                    },
+                    "required": [
+                        "kind",
+                        "persistableReference"
+                    ]
+                }
+            ],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractMetaItem:1.0.0",
+            "description": "A meta data item, which allows the association of named properties or property values to a Unit/Measurement/CRS/Azimuth/Time context.",
+            "title": "Frame of Reference Meta Data Item",
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractMetaItem.1.0.0.json"
+        },
+        "osdu:wks:AbstractLegalParentList:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractLegalParentList:1.0.0",
+            "description": "A list of entity IDs in the data ecosystem, which act as legal parents to the current entity. This structure is included by the SystemProperties \"ancestry\", which is part of all OSDU records. Not extensible.",
+            "additionalProperties": false,
+            "title": "Parent List",
+            "type": "object",
+            "properties": {
+                "parents": {
+                    "description": "An array of none, one or many entity references in the data ecosystem, which identify the source of data in the legal sense. In contract to other relationships, the source record version is required. Example: the 'parents' will be queried when e.g. the subscription of source data services is terminated; access to the derivatives is also terminated.",
+                    "title": "Parents",
+                    "type": "array",
+                    "items": {
+                        "x-osdu-relationship": [],
+                        "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]+$",
+                        "type": "string"
+                    },
+                    "example": []
+                }
+            },
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalParentList.1.0.0.json"
+        },
+        "osdu:wks:reference-data--GenericReferenceData:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:reference-data--GenericReferenceData:1.0.0",
+            "description": "An auto-generated placeholder schema representing reference-data group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.",
+            "title": "GenericReferenceData",
+            "type": "object",
+            "properties": {
+                "ancestry": {
+                    "description": "The links to data, which constitute the inputs.",
+                    "title": "Ancestry",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0"
+                },
+                "data": {
+                    "allOf": [
+                        {
+                            "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0"
+                        }
+                    ]
+                },
+                "kind": {
+                    "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+                    "description": "The schema identification for the OSDU resource object following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.",
+                    "title": "Entity Kind",
+                    "type": "string",
+                    "example": "osdu:wks:reference-data--GenericReferenceData:1.0.0"
+                },
+                "acl": {
+                    "description": "The access control tags associated with this entity.",
+                    "title": "Access Control List",
+                    "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0"
+                },
+                "version": {
+                    "format": "int64",
+                    "description": "The version number of this OSDU resource; set by the framework.",
+                    "title": "Version Number",
+                    "type": "integer",
+                    "example": 1562066009929332
+                },
+                "tags": {
+                    "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.",
+                    "additionalProperties": {
+                        "type": "string"
+                    },
+                    "title": "Tag Dictionary",
+                    "type": "object",
+                    "example": {
+                        "NameOfKey": "String value"
+                    }
+                },
+                "modifyUser": {
+                    "description": "The user reference, which created this version of this resource object. Set by the System.",
+                    "title": "Resource Object Version Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "modifyTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Version Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:52:24.477Z"
+                },
+                "createTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:46:20.163Z"
+                },
+                "meta": {
+                    "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.",
+                    "title": "Frame of Reference Meta Data",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0"
+                    }
+                },
+                "legal": {
+                    "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.",
+                    "title": "Legal Tags",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0"
+                },
+                "createUser": {
+                    "description": "The user reference, which created the first version of this resource object. Set by the System.",
+                    "title": "Resource Object Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "id": {
+                    "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:reference-data\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$",
+                    "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.",
+                    "title": "Entity ID",
+                    "type": "string",
+                    "example": "namespace:reference-data--GenericReferenceData:63ca0ed3-d6fb-53f0-8549-0916ef144266"
+                }
+            },
+            "required": [
+                "id",
+                "kind",
+                "acl",
+                "legal"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/reference-data/GenericReferenceData.1.0.0.json"
+        },
+        "osdu:wks:AbstractFeatureCollection:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractFeatureCollection:1.0.0",
+            "description": "GeoJSON feature collection as originally published in https://geojson.org/schema/FeatureCollection.json. Attention: the coordinate order is fixed: Longitude first, followed by Latitude, optionally height above MSL (EPSG:5714) as third coordinate.",
+            "title": "GeoJSON FeatureCollection",
+            "type": "object",
+            "required": [
+                "type",
+                "features"
+            ],
+            "properties": {
+                "type": {
+                    "type": "string",
+                    "enum": [
+                        "FeatureCollection"
+                    ]
+                },
+                "features": {
+                    "type": "array",
+                    "items": {
+                        "title": "GeoJSON Feature",
+                        "type": "object",
+                        "required": [
+                            "type",
+                            "properties",
+                            "geometry"
+                        ],
+                        "properties": {
+                            "geometry": {
+                                "oneOf": [
+                                    {
+                                        "type": "null"
+                                    },
+                                    {
+                                        "title": "GeoJSON Point",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "minItems": 2,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "Point"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "GeoJSON LineString",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "minItems": 2,
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 2,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "type": "number"
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "LineString"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "GeoJSON Polygon",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 4,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "minItems": 2,
+                                                        "type": "array",
+                                                        "items": {
+                                                            "type": "number"
+                                                        }
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "Polygon"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "GeoJSON MultiPoint",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 2,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "type": "number"
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "MultiPoint"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "GeoJSON MultiLineString",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 2,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "minItems": 2,
+                                                        "type": "array",
+                                                        "items": {
+                                                            "type": "number"
+                                                        }
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "MultiLineString"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "GeoJSON MultiPolygon",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "array",
+                                                    "items": {
+                                                        "minItems": 4,
+                                                        "type": "array",
+                                                        "items": {
+                                                            "minItems": 2,
+                                                            "type": "array",
+                                                            "items": {
+                                                                "type": "number"
+                                                            }
+                                                        }
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "MultiPolygon"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "GeoJSON GeometryCollection",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "geometries"
+                                        ],
+                                        "properties": {
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "GeometryCollection"
+                                                ]
+                                            },
+                                            "geometries": {
+                                                "type": "array",
+                                                "items": {
+                                                    "oneOf": [
+                                                        {
+                                                            "title": "GeoJSON Point",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "minItems": 2,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "Point"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "GeoJSON LineString",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "minItems": 2,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 2,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "type": "number"
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "LineString"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "GeoJSON Polygon",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 4,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "minItems": 2,
+                                                                            "type": "array",
+                                                                            "items": {
+                                                                                "type": "number"
+                                                                            }
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "Polygon"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "GeoJSON MultiPoint",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 2,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "type": "number"
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "MultiPoint"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "GeoJSON MultiLineString",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 2,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "minItems": 2,
+                                                                            "type": "array",
+                                                                            "items": {
+                                                                                "type": "number"
+                                                                            }
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "MultiLineString"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "GeoJSON MultiPolygon",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "minItems": 4,
+                                                                            "type": "array",
+                                                                            "items": {
+                                                                                "minItems": 2,
+                                                                                "type": "array",
+                                                                                "items": {
+                                                                                    "type": "number"
+                                                                                }
+                                                                            }
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "MultiPolygon"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        }
+                                                    ]
+                                                }
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    }
+                                ]
+                            },
+                            "type": {
+                                "type": "string",
+                                "enum": [
+                                    "Feature"
+                                ]
+                            },
+                            "properties": {
+                                "oneOf": [
+                                    {
+                                        "type": "null"
+                                    },
+                                    {
+                                        "type": "object"
+                                    }
+                                ]
+                            },
+                            "bbox": {
+                                "minItems": 4,
+                                "type": "array",
+                                "items": {
+                                    "type": "number"
+                                }
+                            }
+                        }
+                    }
+                },
+                "bbox": {
+                    "minItems": 4,
+                    "type": "array",
+                    "items": {
+                        "type": "number"
+                    }
+                }
+            },
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractFeatureCollection.1.0.0.json"
+        },
+        "osdu:wks:AbstractAnyCrsFeatureCollection:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractAnyCrsFeatureCollection:1.0.0",
+            "description": "A schema like GeoJSON FeatureCollection with a non-WGS 84 CRS context; based on https://geojson.org/schema/FeatureCollection.json. Attention: the coordinate order is fixed: Longitude/Easting/Westing/X first, followed by Latitude/Northing/Southing/Y, optionally height as third coordinate.",
+            "title": "AbstractAnyCrsFeatureCollection",
+            "type": "object",
+            "required": [
+                "type",
+                "persistableReferenceCrs",
+                "features"
+            ],
+            "properties": {
+                "CoordinateReferenceSystemID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-CoordinateReferenceSystem:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "The CRS reference into the CoordinateReferenceSystem catalog.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "CoordinateReferenceSystem",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Coordinate Reference System ID",
+                    "type": "string",
+                    "example": "namespace:reference-data--CoordinateReferenceSystem:BoundCRS.SLB.32021.15851:"
+                },
+                "persistableReferenceCrs": {
+                    "description": "The CRS reference as persistableReference string. If populated, the CoordinateReferenceSystemID takes precedence.",
+                    "type": "string",
+                    "title": "CRS Reference",
+                    "example": "{\"lateBoundCRS\":{\"wkt\":\"PROJCS[\\\"NAD_1927_StatePlane_North_Dakota_South_FIPS_3302\\\",GEOGCS[\\\"GCS_North_American_1927\\\",DATUM[\\\"D_North_American_1927\\\",SPHEROID[\\\"Clarke_1866\\\",6378206.4,294.9786982]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],PROJECTION[\\\"Lambert_Conformal_Conic\\\"],PARAMETER[\\\"False_Easting\\\",2000000.0],PARAMETER[\\\"False_Northing\\\",0.0],PARAMETER[\\\"Central_Meridian\\\",-100.5],PARAMETER[\\\"Standard_Parallel_1\\\",46.1833333333333],PARAMETER[\\\"Standard_Parallel_2\\\",47.4833333333333],PARAMETER[\\\"Latitude_Of_Origin\\\",45.6666666666667],UNIT[\\\"Foot_US\\\",0.304800609601219],AUTHORITY[\\\"EPSG\\\",32021]]\",\"ver\":\"PE_10_3_1\",\"name\":\"NAD_1927_StatePlane_North_Dakota_South_FIPS_3302\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"32021\"},\"type\":\"LBC\"},\"singleCT\":{\"wkt\":\"GEOGTRAN[\\\"NAD_1927_To_WGS_1984_79_CONUS\\\",GEOGCS[\\\"GCS_North_American_1927\\\",DATUM[\\\"D_North_American_1927\\\",SPHEROID[\\\"Clarke_1866\\\",6378206.4,294.9786982]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],GEOGCS[\\\"GCS_WGS_1984\\\",DATUM[\\\"D_WGS_1984\\\",SPHEROID[\\\"WGS_1984\\\",6378137.0,298.257223563]],PRIMEM[\\\"Greenwich\\\",0.0],UNIT[\\\"Degree\\\",0.0174532925199433]],METHOD[\\\"NADCON\\\"],PARAMETER[\\\"Dataset_conus\\\",0.0],AUTHORITY[\\\"EPSG\\\",15851]]\",\"ver\":\"PE_10_3_1\",\"name\":\"NAD_1927_To_WGS_1984_79_CONUS\",\"authCode\":{\"auth\":\"EPSG\",\"code\":\"15851\"},\"type\":\"ST\"},\"ver\":\"PE_10_3_1\",\"name\":\"NAD27 * OGP-Usa Conus / North Dakota South [32021,15851]\",\"authCode\":{\"auth\":\"SLB\",\"code\":\"32021079\"},\"type\":\"EBC\"}"
+                },
+                "features": {
+                    "type": "array",
+                    "items": {
+                        "title": "AnyCrsGeoJSON Feature",
+                        "type": "object",
+                        "required": [
+                            "type",
+                            "properties",
+                            "geometry"
+                        ],
+                        "properties": {
+                            "geometry": {
+                                "oneOf": [
+                                    {
+                                        "type": "null"
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON Point",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "minItems": 2,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsPoint"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON LineString",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "minItems": 2,
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 2,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "type": "number"
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsLineString"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON Polygon",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 4,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "minItems": 2,
+                                                        "type": "array",
+                                                        "items": {
+                                                            "type": "number"
+                                                        }
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsPolygon"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON MultiPoint",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 2,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "type": "number"
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsMultiPoint"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON MultiLineString",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "minItems": 2,
+                                                    "type": "array",
+                                                    "items": {
+                                                        "minItems": 2,
+                                                        "type": "array",
+                                                        "items": {
+                                                            "type": "number"
+                                                        }
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsMultiLineString"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON MultiPolygon",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "coordinates"
+                                        ],
+                                        "properties": {
+                                            "coordinates": {
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "array",
+                                                    "items": {
+                                                        "minItems": 4,
+                                                        "type": "array",
+                                                        "items": {
+                                                            "minItems": 2,
+                                                            "type": "array",
+                                                            "items": {
+                                                                "type": "number"
+                                                            }
+                                                        }
+                                                    }
+                                                }
+                                            },
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsMultiPolygon"
+                                                ]
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    },
+                                    {
+                                        "title": "AnyCrsGeoJSON GeometryCollection",
+                                        "type": "object",
+                                        "required": [
+                                            "type",
+                                            "geometries"
+                                        ],
+                                        "properties": {
+                                            "type": {
+                                                "type": "string",
+                                                "enum": [
+                                                    "AnyCrsGeometryCollection"
+                                                ]
+                                            },
+                                            "geometries": {
+                                                "type": "array",
+                                                "items": {
+                                                    "oneOf": [
+                                                        {
+                                                            "title": "AnyCrsGeoJSON Point",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "minItems": 2,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "AnyCrsPoint"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "AnyCrsGeoJSON LineString",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "minItems": 2,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 2,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "type": "number"
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "AnyCrsLineString"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "AnyCrsGeoJSON Polygon",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 4,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "minItems": 2,
+                                                                            "type": "array",
+                                                                            "items": {
+                                                                                "type": "number"
+                                                                            }
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "AnyCrsPolygon"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "AnyCrsGeoJSON MultiPoint",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 2,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "type": "number"
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "AnyCrsMultiPoint"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "AnyCrsGeoJSON MultiLineString",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "minItems": 2,
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "minItems": 2,
+                                                                            "type": "array",
+                                                                            "items": {
+                                                                                "type": "number"
+                                                                            }
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "AnyCrsMultiLineString"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        },
+                                                        {
+                                                            "title": "AnyCrsGeoJSON MultiPolygon",
+                                                            "type": "object",
+                                                            "required": [
+                                                                "type",
+                                                                "coordinates"
+                                                            ],
+                                                            "properties": {
+                                                                "coordinates": {
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "array",
+                                                                        "items": {
+                                                                            "minItems": 4,
+                                                                            "type": "array",
+                                                                            "items": {
+                                                                                "minItems": 2,
+                                                                                "type": "array",
+                                                                                "items": {
+                                                                                    "type": "number"
+                                                                                }
+                                                                            }
+                                                                        }
+                                                                    }
+                                                                },
+                                                                "type": {
+                                                                    "type": "string",
+                                                                    "enum": [
+                                                                        "AnyCrsMultiPolygon"
+                                                                    ]
+                                                                },
+                                                                "bbox": {
+                                                                    "minItems": 4,
+                                                                    "type": "array",
+                                                                    "items": {
+                                                                        "type": "number"
+                                                                    }
+                                                                }
+                                                            }
+                                                        }
+                                                    ]
+                                                }
+                                            },
+                                            "bbox": {
+                                                "minItems": 4,
+                                                "type": "array",
+                                                "items": {
+                                                    "type": "number"
+                                                }
+                                            }
+                                        }
+                                    }
+                                ]
+                            },
+                            "type": {
+                                "type": "string",
+                                "enum": [
+                                    "AnyCrsFeature"
+                                ]
+                            },
+                            "properties": {
+                                "oneOf": [
+                                    {
+                                        "type": "null"
+                                    },
+                                    {
+                                        "type": "object"
+                                    }
+                                ]
+                            },
+                            "bbox": {
+                                "minItems": 4,
+                                "type": "array",
+                                "items": {
+                                    "type": "number"
+                                }
+                            }
+                        }
+                    }
+                },
+                "persistableReferenceUnitZ": {
+                    "description": "The unit of measure for the Z-axis (only for 3-dimensional coordinates, where the CRS does not describe the vertical unit). Note that the direction is upwards positive, i.e. Z means height.",
+                    "type": "string",
+                    "title": "Z-Unit Reference",
+                    "example": "{\"scaleOffset\":{\"scale\":1.0,\"offset\":0.0},\"symbol\":\"m\",\"baseMeasurement\":{\"ancestry\":\"Length\",\"type\":\"UM\"},\"type\":\"USO\"}"
+                },
+                "bbox": {
+                    "minItems": 4,
+                    "type": "array",
+                    "items": {
+                        "type": "number"
+                    }
+                },
+                "persistableReferenceVerticalCrs": {
+                    "description": "The VerticalCRS reference as persistableReference string. If populated, the VerticalCoordinateReferenceSystemID takes precedence. The property is null or empty for 2D geometries. For 3D geometries and absent or null persistableReferenceVerticalCrs the vertical CRS is either provided via persistableReferenceCrs's CompoundCRS or it is implicitly defined as EPSG:5714 MSL height.",
+                    "type": "string",
+                    "title": "Vertical CRS Reference",
+                    "example": "{\"authCode\":{\"auth\":\"EPSG\",\"code\":\"5773\"},\"type\":\"LBC\",\"ver\":\"PE_10_3_1\",\"name\":\"EGM96_Geoid\",\"wkt\":\"VERTCS[\\\"EGM96_Geoid\\\",VDATUM[\\\"EGM96_Geoid\\\"],PARAMETER[\\\"Vertical_Shift\\\",0.0],PARAMETER[\\\"Direction\\\",1.0],UNIT[\\\"Meter\\\",1.0],AUTHORITY[\\\"EPSG\\\",5773]]\"}"
+                },
+                "type": {
+                    "type": "string",
+                    "enum": [
+                        "AnyCrsFeatureCollection"
+                    ]
+                },
+                "VerticalCoordinateReferenceSystemID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-CoordinateReferenceSystem:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "The explicit VerticalCRS reference into the CoordinateReferenceSystem catalog. This property stays empty for 2D geometries. Absent or empty values for 3D geometries mean the context may be provided by a CompoundCRS in 'CoordinateReferenceSystemID' or implicitly EPSG:5714 MSL height",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "CoordinateReferenceSystem",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "title": "Vertical Coordinate Reference System ID",
+                    "type": "string",
+                    "example": "namespace:reference-data--CoordinateReferenceSystem:VerticalCRS.EPSG.5773:"
+                }
+            },
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAnyCrsFeatureCollection.1.0.0.json"
+        },
+        "osdu:wks:AbstractSpatialLocation:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractSpatialLocation:1.0.0",
+            "description": "A geographic object which can be described by a set of points.",
+            "title": "AbstractSpatialLocation",
+            "type": "object",
+            "properties": {
+                "AsIngestedCoordinates": {
+                    "description": "The original or 'as ingested' coordinates (Point, MultiPoint, LineString, MultiLineString, Polygon or MultiPolygon). The name 'AsIngestedCoordinates' was chosen to contrast it to 'OriginalCoordinates', which carries the uncertainty whether any coordinate operations took place before ingestion. In cases where the original CRS is different from the as-ingested CRS, the OperationsApplied can also contain the list of operations applied to the coordinate prior to ingestion. The data structure is similar to GeoJSON FeatureCollection, however in a CRS context explicitly defined within the AbstractAnyCrsFeatureCollection. The coordinate sequence follows GeoJSON standard, i.e. 'eastward/longitude', 'northward/latitude' {, 'upward/height' unless overridden by an explicit direction in the AsIngestedCoordinates.VerticalCoordinateReferenceSystemID}.",
+                    "x-osdu-frame-of-reference": "CRS:",
+                    "title": "As Ingested Coordinates",
+                    "$ref": "#/definitions/osdu:wks:AbstractAnyCrsFeatureCollection:1.0.0"
+                },
+                "SpatialParameterTypeID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-SpatialParameterType:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "A type of spatial representation of an object, often general (e.g. an Outline, which could be applied to Field, Reservoir, Facility, etc.) or sometimes specific (e.g. Onshore Outline, State Offshore Outline, Federal Offshore Outline, 3 spatial representations that may be used by Countries).",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "SpatialParameterType",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "type": "string"
+                },
+                "QuantitativeAccuracyBandID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-QuantitativeAccuracyBand:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "An approximate quantitative assessment of the quality of a location (accurate to > 500 m (i.e. not very accurate)), to < 1 m, etc.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "QuantitativeAccuracyBand",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "type": "string"
+                },
+                "CoordinateQualityCheckRemarks": {
+                    "type": "array",
+                    "description": "Freetext remarks on Quality Check.",
+                    "items": {
+                        "type": "string"
+                    }
+                },
+                "AppliedOperations": {
+                    "description": "The audit trail of operations applied to the coordinates from the original state to the current state. The list may contain operations applied prior to ingestion as well as the operations applied to produce the Wgs84Coordinates. The text elements refer to ESRI style CRS and Transformation names, which may have to be translated to EPSG standard names.",
+                    "title": "Operations Applied",
+                    "type": "array",
+                    "items": {
+                        "type": "string"
+                    },
+                    "example": [
+                        "conversion from ED_1950_UTM_Zone_31N to GCS_European_1950; 1 points converted",
+                        "transformation GCS_European_1950 to GCS_WGS_1984 using ED_1950_To_WGS_1984_24; 1 points successfully transformed"
+                    ]
+                },
+                "QualitativeSpatialAccuracyTypeID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-QualitativeSpatialAccuracyType:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "A qualitative description of the quality of a spatial location, e.g. unverifiable, not verified, basic validation.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "QualitativeSpatialAccuracyType",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "type": "string"
+                },
+                "CoordinateQualityCheckPerformedBy": {
+                    "type": "string",
+                    "description": "The user who performed the Quality Check."
+                },
+                "SpatialLocationCoordinatesDate": {
+                    "format": "date-time",
+                    "description": "Date when coordinates were measured or retrieved.",
+                    "x-osdu-frame-of-reference": "DateTime",
+                    "type": "string"
+                },
+                "CoordinateQualityCheckDateTime": {
+                    "format": "date-time",
+                    "description": "The date of the Quality Check.",
+                    "x-osdu-frame-of-reference": "DateTime",
+                    "type": "string"
+                },
+                "Wgs84Coordinates": {
+                    "title": "WGS 84 Coordinates",
+                    "description": "The normalized coordinates (Point, MultiPoint, LineString, MultiLineString, Polygon or MultiPolygon) based on WGS 84 (EPSG:4326 for 2-dimensional coordinates, EPSG:4326 + EPSG:5714 (MSL) for 3-dimensional coordinates). This derived coordinate representation is intended for global discoverability only. The schema of this substructure is identical to the GeoJSON FeatureCollection https://geojson.org/schema/FeatureCollection.json. The coordinate sequence follows GeoJSON standard, i.e. longitude, latitude {, height}",
+                    "$ref": "#/definitions/osdu:wks:AbstractFeatureCollection:1.0.0"
+                },
+                "SpatialGeometryTypeID": {
+                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-SpatialGeometryType:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                    "description": "Indicates the expected look of the SpatialParameterType, e.g. Point, MultiPoint, LineString, MultiLineString, Polygon, MultiPolygon. The value constrains the type of geometries in the GeoJSON Wgs84Coordinates and AsIngestedCoordinates.",
+                    "x-osdu-relationship": [
+                        {
+                            "EntityType": "SpatialGeometryType",
+                            "GroupType": "reference-data"
+                        }
+                    ],
+                    "type": "string"
+                }
+            },
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractSpatialLocation.1.0.0.json"
+        },
+        "osdu:wks:dataset--GenericDataset:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:dataset--GenericDataset:1.0.0",
+            "description": "An auto-generated placeholder schema representing dataset group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.",
+            "title": "GenericDataset",
+            "type": "object",
+            "properties": {
+                "ancestry": {
+                    "description": "The links to data, which constitute the inputs.",
+                    "title": "Ancestry",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0"
+                },
+                "data": {
+                    "allOf": [
+                        {
+                            "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0"
+                        },
+                        {
+                            "description": "Schema fragment holding properties common for all datasets.",
+                            "title": "AbstractDataset",
+                            "type": "object",
+                            "properties": {
+                                "Endian": {
+                                    "description": "Endianness of binary value.  Enumeration: \"BIG\", \"LITTLE\".  If absent, applications will need to interpret from context indicators.",
+                                    "type": "string",
+                                    "enum": [
+                                        "BIG",
+                                        "LITTLE"
+                                    ]
+                                },
+                                "Description": {
+                                    "description": "An optional, textual description of the dataset.",
+                                    "type": "string",
+                                    "title": "Description",
+                                    "example": "As originally delivered by ACME.com."
+                                },
+                                "DatasetProperties": {
+                                    "description": "Placeholder for a specialization.",
+                                    "type": "object",
+                                    "title": "Dataset Properties",
+                                    "example": {}
+                                },
+                                "TotalSize": {
+                                    "format": "integer",
+                                    "pattern": "^[0-9]+$",
+                                    "description": "Total size of the dataset in bytes; for files it is the same as declared in FileSourceInfo.FileSize or the sum of all individual files. Implemented as string. The value must be convertible to a long integer (sizes can become very large).",
+                                    "type": "string",
+                                    "title": "Total Size",
+                                    "example": 13245217273
+                                },
+                                "EncodingFormatTypeID": {
+                                    "x-osdu-relationship": [
+                                        {
+                                            "EntityType": "EncodingFormatType",
+                                            "GroupType": "reference-data"
+                                        }
+                                    ],
+                                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-EncodingFormatType:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                                    "description": "EncodingFormatType ID reference value relationship. It can me a mime-type or media-type.",
+                                    "type": "string",
+                                    "title": "Encoding Format Type ID",
+                                    "example": "namespace:reference-data--EncodingFormatType:text%2Fcsv:"
+                                },
+                                "Name": {
+                                    "description": "An optional name of the dataset, e.g. a user friendly file or file collection name.",
+                                    "type": "string",
+                                    "title": "Name",
+                                    "example": "Dataset X221/15"
+                                },
+                                "SchemaFormatTypeID": {
+                                    "x-osdu-relationship": [
+                                        {
+                                            "EntityType": "SchemaFormatType",
+                                            "GroupType": "reference-data"
+                                        }
+                                    ],
+                                    "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-SchemaFormatType:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                                    "description": "Relationship to the SchemaFormatType reference value.",
+                                    "type": "string",
+                                    "title": "Schema Format Type ID",
+                                    "example": "namespace:reference-data--SchemaFormatType:CWLS%20LAS3:"
+                                }
+                            }
+                        }
+                    ]
+                },
+                "kind": {
+                    "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+                    "description": "The schema identification for the OSDU resource object following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.",
+                    "title": "Entity Kind",
+                    "type": "string",
+                    "example": "osdu:wks:dataset--GenericDataset:1.0.0"
+                },
+                "acl": {
+                    "description": "The access control tags associated with this entity.",
+                    "title": "Access Control List",
+                    "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0"
+                },
+                "version": {
+                    "format": "int64",
+                    "description": "The version number of this OSDU resource; set by the framework.",
+                    "title": "Version Number",
+                    "type": "integer",
+                    "example": 1562066009929332
+                },
+                "tags": {
+                    "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.",
+                    "additionalProperties": {
+                        "type": "string"
+                    },
+                    "title": "Tag Dictionary",
+                    "type": "object",
+                    "example": {
+                        "NameOfKey": "String value"
+                    }
+                },
+                "modifyUser": {
+                    "description": "The user reference, which created this version of this resource object. Set by the System.",
+                    "title": "Resource Object Version Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "modifyTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Version Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:52:24.477Z"
+                },
+                "createTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:46:20.163Z"
+                },
+                "meta": {
+                    "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.",
+                    "title": "Frame of Reference Meta Data",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0"
+                    }
+                },
+                "legal": {
+                    "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.",
+                    "title": "Legal Tags",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0"
+                },
+                "createUser": {
+                    "description": "The user reference, which created the first version of this resource object. Set by the System.",
+                    "title": "Resource Object Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "id": {
+                    "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:dataset\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$",
+                    "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.",
+                    "title": "Entity ID",
+                    "type": "string",
+                    "example": "namespace:dataset--GenericDataset:b792625f-5cb1-56c5-9699-eb35259e1f9f"
+                }
+            },
+            "required": [
+                "kind",
+                "acl",
+                "legal"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/dataset/GenericDataset.1.0.0.json"
+        },
+        "osdu:wks:AbstractLegalTags:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractLegalTags:1.0.0",
+            "description": "Legal meta data like legal tags, relevant other countries, legal status. This structure is included by the SystemProperties \"legal\", which is part of all OSDU records. Not extensible.",
+            "additionalProperties": false,
+            "title": "Legal Meta Data",
+            "type": "object",
+            "properties": {
+                "legaltags": {
+                    "description": "The list of legal tags, which resolve to legal properties (like country of origin, export classification code, etc.) and rules with the help of the Compliance Service.",
+                    "title": "Legal Tags",
+                    "type": "array",
+                    "items": {
+                        "type": "string"
+                    }
+                },
+                "otherRelevantDataCountries": {
+                    "description": "The list of other relevant data countries as an array of two-letter country codes, see https://en.wikipedia.org/wiki/ISO_3166-1_alpha-2.",
+                    "title": "Other Relevant Data Countries",
+                    "type": "array",
+                    "items": {
+                        "pattern": "^[A-Z]{2}$",
+                        "type": "string"
+                    }
+                },
+                "status": {
+                    "pattern": "^(compliant|uncompliant)$",
+                    "description": "The legal status. Set by the system after evaluation against the compliance rules associated with the \"legaltags\" using the Compliance Service.",
+                    "title": "Legal Status",
+                    "type": "string"
+                }
+            },
+            "required": [
+                "legaltags",
+                "otherRelevantDataCountries"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractLegalTags.1.0.0.json"
+        },
+        "osdu:wks:master-data--GenericMasterData:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:master-data--GenericMasterData:1.0.0",
+            "description": "An auto-generated placeholder schema representing master-data group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.",
+            "title": "GenericMasterData",
+            "type": "object",
+            "properties": {
+                "ancestry": {
+                    "description": "The links to data, which constitute the inputs.",
+                    "title": "Ancestry",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0"
+                },
+                "data": {
+                    "allOf": [
+                        {
+                            "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0"
+                        }
+                    ]
+                },
+                "kind": {
+                    "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+                    "description": "The schema identification for the OSDU resource object following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.",
+                    "title": "Entity Kind",
+                    "type": "string",
+                    "example": "osdu:wks:master-data--GenericMasterData:1.0.0"
+                },
+                "acl": {
+                    "description": "The access control tags associated with this entity.",
+                    "title": "Access Control List",
+                    "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0"
+                },
+                "version": {
+                    "format": "int64",
+                    "description": "The version number of this OSDU resource; set by the framework.",
+                    "title": "Version Number",
+                    "type": "integer",
+                    "example": 1562066009929332
+                },
+                "tags": {
+                    "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.",
+                    "additionalProperties": {
+                        "type": "string"
+                    },
+                    "title": "Tag Dictionary",
+                    "type": "object",
+                    "example": {
+                        "NameOfKey": "String value"
+                    }
+                },
+                "modifyUser": {
+                    "description": "The user reference, which created this version of this resource object. Set by the System.",
+                    "title": "Resource Object Version Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "modifyTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Version Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:52:24.477Z"
+                },
+                "createTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:46:20.163Z"
+                },
+                "meta": {
+                    "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.",
+                    "title": "Frame of Reference Meta Data",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0"
+                    }
+                },
+                "legal": {
+                    "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.",
+                    "title": "Legal Tags",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0"
+                },
+                "createUser": {
+                    "description": "The user reference, which created the first version of this resource object. Set by the System.",
+                    "title": "Resource Object Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "id": {
+                    "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:master-data\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$",
+                    "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.",
+                    "title": "Entity ID",
+                    "type": "string",
+                    "example": "namespace:master-data--GenericMasterData:9ca8054c-bce6-5a3a-b51d-f216fb1085a5"
+                }
+            },
+            "required": [
+                "id",
+                "kind",
+                "acl",
+                "legal"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/master-data/GenericMasterData.1.0.0.json"
+        },
+        "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0",
+            "description": "An auto-generated placeholder schema representing work-product-component group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.",
+            "title": "GenericWorkProductComponent",
+            "type": "object",
+            "properties": {
+                "ancestry": {
+                    "description": "The links to data, which constitute the inputs.",
+                    "title": "Ancestry",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0"
+                },
+                "data": {
+                    "allOf": [
+                        {
+                            "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0"
+                        },
+                        {
+                            "description": "Generic reference object containing the universal group-type properties of a Work Product Component for inclusion in data type specific Work Product Component objects",
+                            "title": "AbstractWPCGroupType",
+                            "type": "object",
+                            "properties": {
+                                "Datasets": {
+                                    "type": "array",
+                                    "items": {
+                                        "x-osdu-relationship": [
+                                            {
+                                                "GroupType": "dataset"
+                                            }
+                                        ],
+                                        "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:dataset\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*)$",
+                                        "description": "The SRN which identifies this OSDU File resource.",
+                                        "type": "string"
+                                    }
+                                },
+                                "IsDiscoverable": {
+                                    "description": "A flag that indicates if the work product component is searchable, which means covered in the search index.",
+                                    "type": "boolean"
+                                },
+                                "Artefacts": {
+                                    "description": "An array of Artefacts - each artefact has a Role, Resource tuple. An artefact is distinct from the file, in the sense certain valuable information is generated during loading process (Artefact generation process). Examples include retrieving location data, performing an OCR which may result in the generation of artefacts which need to be preserved distinctly",
+                                    "type": "array",
+                                    "items": {
+                                        "type": "object",
+                                        "properties": {
+                                            "ResourceID": {
+                                                "x-osdu-relationship": [
+                                                    {
+                                                        "GroupType": "dataset"
+                                                    }
+                                                ],
+                                                "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:dataset\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*)$",
+                                                "description": "The SRN which identifies this OSDU Artefact resource.",
+                                                "type": "string"
+                                            },
+                                            "ResourceKind": {
+                                                "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+                                                "description": "The kind or schema ID of the artefact. Resolvable with the Schema Service.",
+                                                "type": "string"
+                                            },
+                                            "RoleID": {
+                                                "x-osdu-relationship": [
+                                                    {
+                                                        "EntityType": "ArtefactRole",
+                                                        "GroupType": "reference-data"
+                                                    }
+                                                ],
+                                                "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-ArtefactRole:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                                                "description": "The SRN of this artefact's role.",
+                                                "type": "string"
+                                            }
+                                        }
+                                    }
+                                },
+                                "IsExtendedLoad": {
+                                    "description": "A flag that indicates if the work product component is undergoing an extended load.  It reflects the fact that the work product component is in an early stage and may be updated before finalization.",
+                                    "type": "boolean"
+                                }
+                            }
+                        }
+                    ]
+                },
+                "kind": {
+                    "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+                    "description": "The schema identification for the OSDU resource object following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.",
+                    "title": "Entity Kind",
+                    "type": "string",
+                    "example": "osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0"
+                },
+                "acl": {
+                    "description": "The access control tags associated with this entity.",
+                    "title": "Access Control List",
+                    "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0"
+                },
+                "version": {
+                    "format": "int64",
+                    "description": "The version number of this OSDU resource; set by the framework.",
+                    "title": "Version Number",
+                    "type": "integer",
+                    "example": 1562066009929332
+                },
+                "tags": {
+                    "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.",
+                    "additionalProperties": {
+                        "type": "string"
+                    },
+                    "title": "Tag Dictionary",
+                    "type": "object",
+                    "example": {
+                        "NameOfKey": "String value"
+                    }
+                },
+                "modifyUser": {
+                    "description": "The user reference, which created this version of this resource object. Set by the System.",
+                    "title": "Resource Object Version Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "modifyTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Version Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:52:24.477Z"
+                },
+                "createTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:46:20.163Z"
+                },
+                "meta": {
+                    "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.",
+                    "title": "Frame of Reference Meta Data",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0"
+                    }
+                },
+                "legal": {
+                    "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.",
+                    "title": "Legal Tags",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0"
+                },
+                "createUser": {
+                    "description": "The user reference, which created the first version of this resource object. Set by the System.",
+                    "title": "Resource Object Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "id": {
+                    "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:work-product-component\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$",
+                    "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.",
+                    "title": "Entity ID",
+                    "type": "string",
+                    "example": "namespace:work-product-component--GenericWorkProductComponent:80575f9c-fc7b-516b-b44f-996874b9f775"
+                }
+            },
+            "required": [
+                "kind",
+                "acl",
+                "legal"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/work-product-component/GenericWorkProductComponent.1.0.0.json"
+        },
+        "osdu:wks:work-product--GenericWorkProduct:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:work-product--GenericWorkProduct:1.0.0",
+            "description": "An auto-generated placeholder schema representing work-product group-type records in data loading/ingestion/creation manifests. Do not use this kind for actual records.",
+            "title": "GenericWorkProduct",
+            "type": "object",
+            "properties": {
+                "ancestry": {
+                    "description": "The links to data, which constitute the inputs.",
+                    "title": "Ancestry",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalParentList:1.0.0"
+                },
+                "data": {
+                    "allOf": [
+                        {
+                            "$ref": "#/definitions/osdu:wks:AbstractCommonResources:1.0.0"
+                        },
+                        {
+                            "description": "A collection of work product components such as might be produced by a business activity and which is delivered to the data platform for loading.",
+                            "title": "WorkProduct",
+                            "type": "object",
+                            "properties": {
+                                "Description": {
+                                    "description": "Description of the purpose of the work product.",
+                                    "type": "string"
+                                },
+                                "AuthorIDs": {
+                                    "description": "Array of Authors' names of the work product.  Could be a person or company entity.",
+                                    "type": "array",
+                                    "items": {
+                                        "type": "string"
+                                    }
+                                },
+                                "IsExtendedLoad": {
+                                    "description": "A flag that indicates if the work product is undergoing an extended load.  It reflects the fact that the work product is in an early stage and may be updated before finalization.",
+                                    "type": "boolean"
+                                },
+                                "Name": {
+                                    "description": "Name of the instance of Work Product - could be a shipment number.",
+                                    "type": "string"
+                                },
+                                "Components": {
+                                    "type": "array",
+                                    "items": {
+                                        "x-osdu-relationship": [
+                                            {
+                                                "GroupType": "work-product-component"
+                                            }
+                                        ],
+                                        "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:work-product-component\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*)$",
+                                        "description": "The SRN which identifies this OSDU Work Product Component resource.",
+                                        "type": "string"
+                                    }
+                                },
+                                "SpatialArea": {
+                                    "description": "A polygon boundary that reflects the locale of the content of the work product (location of the subject matter).",
+                                    "$ref": "#/definitions/osdu:wks:AbstractSpatialLocation:1.0.0"
+                                },
+                                "CreationDateTime": {
+                                    "format": "date-time",
+                                    "description": "Date that a resource (work  product here) is formed outside of OSDU before loading (e.g. publication date, work product delivery package assembly date).",
+                                    "type": "string"
+                                },
+                                "Annotations": {
+                                    "description": "Array of Annotations",
+                                    "type": "array",
+                                    "items": {
+                                        "type": "string"
+                                    }
+                                },
+                                "BusinessActivities": {
+                                    "description": "Array of business processes/workflows that the work product has been through (ex. well planning, exploration).",
+                                    "type": "array",
+                                    "items": {
+                                        "description": "Business Activity",
+                                        "type": "string"
+                                    }
+                                },
+                                "IsDiscoverable": {
+                                    "description": "A flag that indicates if the work product is searchable, which means covered in the search index.",
+                                    "type": "boolean"
+                                },
+                                "SpatialPoint": {
+                                    "description": "A centroid point that reflects the locale of the content of the work product (location of the subject matter).",
+                                    "$ref": "#/definitions/osdu:wks:AbstractSpatialLocation:1.0.0"
+                                },
+                                "SubmitterName": {
+                                    "description": "Name of the person that first submitted the work product package to OSDU.",
+                                    "type": "string"
+                                },
+                                "LineageAssertions": {
+                                    "description": "Defines relationships with other objects (any kind of Resource) upon which this work product depends.  The assertion is directed only from the asserting WP to ancestor objects, not children.  It should not be used to refer to files or artefacts within the WP -- the association within the WP is sufficient and Artefacts are actually children of the main WP file. They should be recorded in the Data.Artefacts[] array.",
+                                    "type": "array",
+                                    "items": {
+                                        "type": "object",
+                                        "title": "LineageAssertion",
+                                        "properties": {
+                                            "ID": {
+                                                "x-osdu-relationship": [],
+                                                "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                                                "description": "The object reference identifying the DIRECT, INDIRECT, REFERENCE dependency.",
+                                                "type": "string"
+                                            },
+                                            "LineageRelationshipType": {
+                                                "x-osdu-relationship": [
+                                                    {
+                                                        "EntityType": "LineageRelationshipType",
+                                                        "GroupType": "reference-data"
+                                                    }
+                                                ],
+                                                "pattern": "^[\\w\\-\\.]+:reference-data\\-\\-LineageRelationshipType:[\\w\\-\\.\\:\\%]+:[0-9]*$",
+                                                "description": "Used by LineageAssertion to describe the nature of the line of descent of a work product from a prior Resource, such as DIRECT, INDIRECT, REFERENCE.  It is not for proximity (number of nodes away), it is not to cover all the relationships in a full ontology or graph, and it is not to describe the type of activity that created the asserting WP.  LineageAssertion does not encompass a full provenance, process history, or activity model.",
+                                                "type": "string"
+                                            }
+                                        }
+                                    }
+                                },
+                                "Tags": {
+                                    "description": "Array of key words to identify the work product, especially to help in search.",
+                                    "type": "array",
+                                    "items": {
+                                        "type": "string"
+                                    }
+                                }
+                            }
+                        }
+                    ]
+                },
+                "kind": {
+                    "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+                    "description": "The schema identification for the OSDU resource object following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.",
+                    "title": "Entity Kind",
+                    "type": "string",
+                    "example": "osdu:wks:work-product--GenericWorkProduct:1.0.0"
+                },
+                "acl": {
+                    "description": "The access control tags associated with this entity.",
+                    "title": "Access Control List",
+                    "$ref": "#/definitions/osdu:wks:AbstractAccessControlList:1.0.0"
+                },
+                "version": {
+                    "format": "int64",
+                    "description": "The version number of this OSDU resource; set by the framework.",
+                    "title": "Version Number",
+                    "type": "integer",
+                    "example": 1562066009929332
+                },
+                "tags": {
+                    "description": "A generic dictionary of string keys mapping to string value. Only strings are permitted as keys and values.",
+                    "additionalProperties": {
+                        "type": "string"
+                    },
+                    "title": "Tag Dictionary",
+                    "type": "object",
+                    "example": {
+                        "NameOfKey": "String value"
+                    }
+                },
+                "modifyUser": {
+                    "description": "The user reference, which created this version of this resource object. Set by the System.",
+                    "title": "Resource Object Version Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "modifyTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which this version of the OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Version Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:52:24.477Z"
+                },
+                "createTime": {
+                    "format": "date-time",
+                    "description": "Timestamp of the time at which initial version of this OSDU resource object was created. Set by the System. The value is a combined date-time string in ISO-8601 given in UTC.",
+                    "title": "Resource Object Creation DateTime",
+                    "type": "string",
+                    "example": "2020-12-16T11:46:20.163Z"
+                },
+                "meta": {
+                    "description": "The Frame of Reference meta data section linking the named properties to self-contained definitions.",
+                    "title": "Frame of Reference Meta Data",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:AbstractMetaItem:1.0.0"
+                    }
+                },
+                "legal": {
+                    "description": "The entity's legal tags and compliance status. The actual contents associated with the legal tags is managed by the Compliance Service.",
+                    "title": "Legal Tags",
+                    "$ref": "#/definitions/osdu:wks:AbstractLegalTags:1.0.0"
+                },
+                "createUser": {
+                    "description": "The user reference, which created the first version of this resource object. Set by the System.",
+                    "title": "Resource Object Creation User Reference",
+                    "type": "string",
+                    "example": "some-user@some-company-cloud.com"
+                },
+                "id": {
+                    "pattern": "^(surrogate-key:.+|[\\w\\-\\.]+:work-product\\-\\-[\\w\\-\\.]+:[\\w\\-\\.\\:\\%]+)$",
+                    "description": "Previously called ResourceID or SRN which identifies this OSDU resource object without version.",
+                    "title": "Entity ID",
+                    "type": "string",
+                    "example": "namespace:work-product--GenericWorkProduct:955930ee-e6bd-5ae0-a6ee-67ba902e1635"
+                }
+            },
+            "required": [
+                "kind",
+                "acl",
+                "legal"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/work-product/GenericWorkProduct.1.0.0.json"
+        },
+        "osdu:wks:AbstractAccessControlList:1.0.0": {
+            "x-osdu-inheriting-from-kind": [],
+            "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.",
+            "$schema": "http://json-schema.org/draft-07/schema#",
+            "x-osdu-schema-source": "osdu:wks:AbstractAccessControlList:1.0.0",
+            "description": "The access control tags associated with this entity. This structure is included by the SystemProperties \"acl\", which is part of all OSDU records. Not extensible.",
+            "additionalProperties": false,
+            "title": "Access Control List",
+            "type": "object",
+            "properties": {
+                "viewers": {
+                    "description": "The list of viewers to which this data record is accessible/visible/discoverable formatted as an email (core.common.model.storage.validation.ValidationDoc.EMAIL_REGEX).",
+                    "title": "List of Viewers",
+                    "type": "array",
+                    "items": {
+                        "pattern": "^[a-zA-Z0-9_+&*-]+(?:\\.[a-zA-Z0-9_+&*-]+)*@(?:[a-zA-Z0-9-]+\\.)+[a-zA-Z]{2,7}$",
+                        "type": "string"
+                    }
+                },
+                "owners": {
+                    "description": "The list of owners of this data record formatted as an email (core.common.model.storage.validation.ValidationDoc.EMAIL_REGEX).",
+                    "title": "List of Owners",
+                    "type": "array",
+                    "items": {
+                        "pattern": "^[a-zA-Z0-9_+&*-]+(?:\\.[a-zA-Z0-9_+&*-]+)*@(?:[a-zA-Z0-9-]+\\.)+[a-zA-Z]{2,7}$",
+                        "type": "string"
+                    }
+                }
+            },
+            "required": [
+                "owners",
+                "viewers"
+            ],
+            "$id": "https://schema.osdu.opengroup.org/json/abstract/AbstractAccessControlList.1.0.0.json"
+        }
+    },
+    "properties": {
+        "ReferenceData": {
+            "description": "Reference-data are submitted as an array of records.",
+            "type": "array",
+            "items": {
+                "$ref": "#/definitions/osdu:wks:reference-data--GenericReferenceData:1.0.0"
+            }
+        },
+        "MasterData": {
+            "description": "Master-data are submitted as an array of records.",
+            "type": "array",
+            "items": {
+                "$ref": "#/definitions/osdu:wks:master-data--GenericMasterData:1.0.0"
+            }
+        },
+        "kind": {
+            "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$",
+            "description": "The schema identification for the manifest record following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.",
+            "title": "Manifest  Kind",
+            "type": "string",
+            "example": "osdu:wks:Manifest:1.0.0"
+        },
+        "Data": {
+            "description": "Manifest schema for work-product, work-product-component, dataset ensembles. The items in 'Datasets' are processed first since they are referenced by 'WorkProductComponents' ('data.DatasetIDs[]' and 'data.Artefacts[].ResourceID'). The WorkProduct is processed last collecting the WorkProductComponents.",
+            "type": "object",
+            "properties": {
+                "WorkProduct": {
+                    "description": "The work-product component capturing the work-product-component records belonging to this loading/ingestion transaction.",
+                    "$ref": "#/definitions/osdu:wks:work-product--GenericWorkProduct:1.0.0"
+                },
+                "Datasets": {
+                    "description": "The list of 'Datasets' or data containers holding the actual data. The record ids are usually internal surrogate keys enabling the association of dataset records with work-product-component records, namely via 'DatasetIDs' and 'Artefacts.ResourceID' (both referring to 'dataset' group-type entity types).",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:dataset--GenericDataset:1.0.0"
+                    }
+                },
+                "WorkProductComponents": {
+                    "description": "The list of work-product-components records. The record ids are internal surrogate keys enabling the association of work-product-component records with the work-product records.",
+                    "type": "array",
+                    "items": {
+                        "$ref": "#/definitions/osdu:wks:work-product-component--GenericWorkProductComponent:1.0.0"
+                    }
+                }
+            }
+        }
+    },
+    "$id": "https://schema.osdu.opengroup.org/json/manifest/Manifest.1.0.0.json"
+}
diff --git a/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json
new file mode 100644
index 0000000000000000000000000000000000000000..b88045eb90fc2b6f5813f70ce508535592ab4cbc
--- /dev/null
+++ b/tests/plugin-unit-tests/data/master/batch_Wellbore.0.3.0.json
@@ -0,0 +1,1177 @@
+{
+    "execution_context": {
+        "Payload": {
+            "authorization": "Bearer test",
+            "data-partition-id": "opendes",
+            "AppKey": "",
+            "kind_version": "3.0.0"
+        },
+        "$schema": "https://schema.osdu.opengroup.org/json/master-data/Wellbore.1.0.0.json",
+        "$filename": "load_Wellbore.1.0.0_350112350400.json",
+        "manifest": [
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            },
+            {
+                "kind": "test:test:Manifest:1.0.0",
+                "ReferenceData": [],
+                "MasterData": [
+                    {
+                        "id": "opendes:master-data/Wellbore:350112350400",
+                        "kind": "opendes:osdu:TestMaster:0.3.0",
+                        "groupType": "master-data",
+                        "version": 1,
+                        "acl": {
+                            "owners": [
+                                "data.default.viewers@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ],
+                            "viewers": [
+                                "data.default.owners@opendes.osdu-gcp.go3-nrg.projects.epam.com"
+                            ]
+                        },
+                        "legal": {
+                            "legaltags": [
+                                "opendes-demo-legaltag"
+                            ],
+                            "otherRelevantDataCountries": [
+                                "srn:opendes:master-data/GeoPoliticalEntity:USA:"
+                            ],
+                            "status": "srn:opendes:reference-data/LegalStatus:public:1111"
+                        },
+                        "resourceHostRegionIDs": [
+                            "srn:opendes:reference-data/OSDURegion:US-EAST:"
+                        ],
+                        "resourceObjectCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceVersionCreationDateTime": "2020-10-16T11:14:45-05:00",
+                        "resourceSecurityClassification": "srn:opendes:reference-data/ResourceSecurityClassification:public:",
+                        "source": "srn:opendes:master-data/Organisation:Oklahoma Corporation Commission:",
+                        "existenceKind": "srn:opendes:reference-data/ExistenceKind:Active:",
+                        "licenseState": "srn:opendes:reference-data/LicenseState:Unlicensed:",
+                        "data": {
+                            "SequenceNumber": 1
+                        },
+                        "schema": "test:test:GenericMasterData:1.0.0"
+                    }
+                ],
+                "Data": {}
+            }
+        ]
+    },
+    "workflow_name": "osdu_ingest",
+    "run_id": "foo"
+}
diff --git a/tests/plugin-unit-tests/data/master/r3_Well.json b/tests/plugin-unit-tests/data/master/r3_Well.json
new file mode 100644
index 0000000000000000000000000000000000000000..f65e5603835f8bb6dce12fc2c04da4547949ce2a
--- /dev/null
+++ b/tests/plugin-unit-tests/data/master/r3_Well.json
@@ -0,0 +1,72 @@
+{
+    "MasterData": [
+        {
+            "id": "osdu:master-data--Well:BA050018",
+            "kind": "osdu:wks:master-data--Well:1.0.0",
+            "acl": {
+                "owners": [
+                    "ownergroup@testcompany.com"
+                ],
+                "viewers": [
+                    "viewgroup@testcompany.com"
+                ]
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:Public:",
+                "Source": "NL_TNO",
+                "SpatialLocation": {
+                    "Wgs84Coordinates": {
+                        "type": "FeatureCollection",
+                        "features": [
+                            {
+                                "type": "Feature",
+                                "geometry": {
+                                    "type": "Point",
+                                    "coordinates": [
+                                        3.51906683,
+                                        55.68101428
+                                    ]
+                                },
+                                "properties": {}
+                            }
+                        ]
+                    }
+                },
+                "FacilityID": "10110909",
+                "FacilityTypeID": "osdu:reference-data--FacilityType:WELL:",
+                "FacilityOperator": [
+                    {
+                        "FacilityOperatorID": "410464",
+                        "FacilityOperatorOrganisationID": "osdu:master-data--Organisation:HESS:"
+                    }
+                ],
+                "FacilityName": "A05-01",
+                "FacilityNameAlias": [
+                    {
+                        "AliasName": "A05-01",
+                        "AliasNameTypeID": "osdu:reference-data--AliasNameType:WELL_NAME:"
+                    }
+                ],
+                "FacilityEvent": [
+                    {
+                        "FacilityEventTypeID": "osdu:reference-data--FacilityEventType:SPUD_DATE:",
+                        "EffectiveDateTime": "1999-06-03T00:00:00"
+                    }
+                ],
+                "VerticalMeasurements": [
+                    {
+                        "VerticalMeasurementID": "Kelly Bushing",
+                        "VerticalMeasurement": 36.6,
+                        "VerticalMeasurementPathID": "osdu:reference-data--VerticalMeasurementPath:DEPTH_DATUM_ELEV:"
+                    }
+                ],
+                "NameAliases": [],
+                "GeoContexts": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            }
+        }
+    ]
+}
diff --git a/tests/plugin-unit-tests/data/master/ref_result_r3_Well.json b/tests/plugin-unit-tests/data/master/ref_result_r3_Well.json
new file mode 100644
index 0000000000000000000000000000000000000000..547319536e9b81ad75995e8e45e112c3111e96ed
--- /dev/null
+++ b/tests/plugin-unit-tests/data/master/ref_result_r3_Well.json
@@ -0,0 +1,8 @@
+[
+    "osdu:reference-data--ResourceSecurityClassification:Public",
+    "osdu:reference-data--FacilityType:WELL",
+    "osdu:master-data--Organisation:HESS",
+    "osdu:reference-data--AliasNameType:WELL_NAME",
+    "osdu:reference-data--FacilityEventType:SPUD_DATE",
+    "osdu:reference-data--VerticalMeasurementPath:DEPTH_DATUM_ELEV"
+]
diff --git a/tests/plugin-unit-tests/data/master/traversal_r3_Well.json b/tests/plugin-unit-tests/data/master/traversal_r3_Well.json
new file mode 100644
index 0000000000000000000000000000000000000000..26db0a3849392f65f99518f5122de68913bf1f95
--- /dev/null
+++ b/tests/plugin-unit-tests/data/master/traversal_r3_Well.json
@@ -0,0 +1,74 @@
+[
+    {
+        "entity": {
+            "id": "osdu:master-data--Well:BA050018",
+            "kind": "osdu:wks:master-data--Well:1.0.0",
+            "acl": {
+                "owners": [
+                    "ownergroup@testcompany.com"
+                ],
+                "viewers": [
+                    "viewgroup@testcompany.com"
+                ]
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:Public:",
+                "Source": "NL_TNO",
+                "SpatialLocation": {
+                    "Wgs84Coordinates": {
+                        "type": "FeatureCollection",
+                        "features": [
+                            {
+                                "type": "Feature",
+                                "geometry": {
+                                    "type": "Point",
+                                    "coordinates": [
+                                        3.51906683,
+                                        55.68101428
+                                    ]
+                                },
+                                "properties": {}
+                            }
+                        ]
+                    }
+                },
+                "FacilityID": "10110909",
+                "FacilityTypeID": "osdu:reference-data--FacilityType:WELL:",
+                "FacilityOperator": [
+                    {
+                        "FacilityOperatorID": "410464",
+                        "FacilityOperatorOrganisationID": "osdu:master-data--Organisation:HESS:"
+                    }
+                ],
+                "FacilityName": "A05-01",
+                "FacilityNameAlias": [
+                    {
+                        "AliasName": "A05-01",
+                        "AliasNameTypeID": "osdu:reference-data--AliasNameType:WELL_NAME:"
+                    }
+                ],
+                "FacilityEvent": [
+                    {
+                        "FacilityEventTypeID": "osdu:reference-data--FacilityEventType:SPUD_DATE:",
+                        "EffectiveDateTime": "1999-06-03T00:00:00"
+                    }
+                ],
+                "VerticalMeasurements": [
+                    {
+                        "VerticalMeasurementID": "Kelly Bushing",
+                        "VerticalMeasurement": 36.6,
+                        "VerticalMeasurementPathID": "osdu:reference-data--VerticalMeasurementPath:DEPTH_DATUM_ELEV:"
+                    }
+                ],
+                "NameAliases": [],
+                "GeoContexts": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            }
+        },
+        "schema": "GenericMasterData.1.0.0.json",
+        "parent":
+    }
+]
diff --git a/tests/plugin-unit-tests/data/other/ExtractedIds.json b/tests/plugin-unit-tests/data/other/ExtractedIds.json
new file mode 100644
index 0000000000000000000000000000000000000000..bc0cd34b713e47b27b5a325825088aa00648bebe
--- /dev/null
+++ b/tests/plugin-unit-tests/data/other/ExtractedIds.json
@@ -0,0 +1,4 @@
+[
+    "some_test_id",
+    "some_test_id:12345"
+]
diff --git a/tests/plugin-unit-tests/data/other/SearchResponseValid.json b/tests/plugin-unit-tests/data/other/SearchResponseValid.json
index 6e8fe92db01ccb71b541a6f204b45499414ec80f..a8303d36fec5cbea37b27a7e089acb82b3563b25 100644
--- a/tests/plugin-unit-tests/data/other/SearchResponseValid.json
+++ b/tests/plugin-unit-tests/data/other/SearchResponseValid.json
@@ -1,5 +1,9 @@
 {
     "results": [
+        {
+            "id": "some_test_id",
+            "version": 12345
+        }
     ],
     "aggregations": null,
     "totalCount": 45
diff --git a/tests/plugin-unit-tests/data/surrogate/Wellbore.0.3.0.json b/tests/plugin-unit-tests/data/surrogate/Wellbore.0.3.0.json
new file mode 100644
index 0000000000000000000000000000000000000000..05affcc7b667677bee3ed8b6d5bc575c91bb8e1a
--- /dev/null
+++ b/tests/plugin-unit-tests/data/surrogate/Wellbore.0.3.0.json
@@ -0,0 +1,166 @@
+[
+    {
+        "id": "surrogate-key:file-1",
+        "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:TabSeparatedColumnarText:",
+            "DatasetProperties": {
+                "FileSourceInfo": {
+                    "FileSource": "",
+                    "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/markers/7587.csv"
+                }
+            }
+        }
+    },
+    {
+        "id": "surrogate-key:wpc-1",
+        "kind": "osdu:wks:work-product-component--WellboreMarker:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "Name": "7587.csv",
+            "Description": "Wellbore Marker",
+            "Datasets": [
+                "surrogate-key:file-1"
+            ],
+            "WellboreID": "osdu:master-data--Wellbore:7587:",
+            "Markers": [
+                {
+                    "MarkerName": "North Sea Supergroup",
+                    "MarkerMeasuredDepth": 0.0
+                },
+                {
+                    "MarkerName": "Ommelanden Formation",
+                    "MarkerMeasuredDepth": 1555.0
+                },
+                {
+                    "MarkerName": "Texel Marlstone Member",
+                    "MarkerMeasuredDepth": 2512.5
+                },
+                {
+                    "MarkerName": "Upper Holland Marl Member",
+                    "MarkerMeasuredDepth": 2606.0
+                },
+                {
+                    "MarkerName": "Middle Holland Claystone Member",
+                    "MarkerMeasuredDepth": 2723.0
+                },
+                {
+                    "MarkerName": "Vlieland Claystone Formation",
+                    "MarkerMeasuredDepth": 2758.0
+                },
+                {
+                    "MarkerName": "Lower Volpriehausen Sandstone Member",
+                    "MarkerMeasuredDepth": 2977.5
+                },
+                {
+                    "MarkerName": "Rogenstein Member",
+                    "MarkerMeasuredDepth": 3018.0
+                },
+                {
+                    "MarkerName": "FAULT",
+                    "MarkerMeasuredDepth": 3043.0
+                },
+                {
+                    "MarkerName": "Upper Zechstein salt",
+                    "MarkerMeasuredDepth": 3043.0
+                },
+                {
+                    "MarkerName": "FAULT",
+                    "MarkerMeasuredDepth": 3544.0
+                },
+                {
+                    "MarkerName": "Z3 Carbonate Member",
+                    "MarkerMeasuredDepth": 3544.0
+                },
+                {
+                    "MarkerName": "Z3 Main Anhydrite Member",
+                    "MarkerMeasuredDepth": 3587.0
+                },
+                {
+                    "MarkerName": "FAULT",
+                    "MarkerMeasuredDepth": 3622.0
+                },
+                {
+                    "MarkerName": "Z3 Salt Member",
+                    "MarkerMeasuredDepth": 3622.0
+                },
+                {
+                    "MarkerName": "Z3 Main Anhydrite Member",
+                    "MarkerMeasuredDepth": 3666.5
+                },
+                {
+                    "MarkerName": "Z3 Carbonate Member",
+                    "MarkerMeasuredDepth": 3688.0
+                },
+                {
+                    "MarkerName": "Z2 Salt Member",
+                    "MarkerMeasuredDepth": 3709.0
+                },
+                {
+                    "MarkerName": "Z2 Basal Anhydrite Member",
+                    "MarkerMeasuredDepth": 3985.0
+                },
+                {
+                    "MarkerName": "Z2 Carbonate Member",
+                    "MarkerMeasuredDepth": 3996.0
+                },
+                {
+                    "MarkerName": "Z1 (Werra) Formation",
+                    "MarkerMeasuredDepth": 4022.5
+                },
+                {
+                    "MarkerName": "Ten Boer Member",
+                    "MarkerMeasuredDepth": 4070.0
+                },
+                {
+                    "MarkerName": "Upper Slochteren Member",
+                    "MarkerMeasuredDepth": 4128.5
+                },
+                {
+                    "MarkerName": "Ameland Member",
+                    "MarkerMeasuredDepth": 4231.0
+                },
+                {
+                    "MarkerName": "Lower Slochteren Member",
+                    "MarkerMeasuredDepth": 4283.5
+                }
+            ]
+        }
+    },
+    {
+        "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+        "acl": {
+            "owners": [],
+            "viewers": []
+        },
+        "legal": {
+            "legaltags": [],
+            "otherRelevantDataCountries": []
+        },
+        "data": {
+            "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+            "Name": "7587.csv",
+            "Description": "Wellbore Marker",
+            "Components": [
+                "surrogate-key:wpc-1"
+            ]
+        }
+    }
+]
diff --git a/tests/plugin-unit-tests/data/workProduct/r3_Welllog.json b/tests/plugin-unit-tests/data/workProduct/r3_Welllog.json
new file mode 100644
index 0000000000000000000000000000000000000000..e32fb1e59373b22ba6a0293fa6d6955d7ed06779
--- /dev/null
+++ b/tests/plugin-unit-tests/data/workProduct/r3_Welllog.json
@@ -0,0 +1,141 @@
+{
+    "kind": "osdu:wks:Manifest:1.0.0",
+    "ReferenceData": [],
+    "MasterData": [],
+    "Data": {
+        "WorkProduct": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "AKM-11 LOG",
+                "Description": "Well Log",
+                "Components": [
+                    "surrogate-key:wpc-1"
+                ]
+            }
+        },
+        "WorkProductComponents": [
+            {
+                "id": "surrogate-key:wpc-1",
+                "kind": "osdu:wks:work-product-component--WellLog:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "Name": "AKM-11 LOG",
+                    "Description": "Well Log",
+                    "Datasets": [
+                        "surrogate-key:file-1"
+                    ],
+                    "WellboreID": "osdu:master-data--Wellbore:1013:",
+                    "TopMeasuredDepth": 2182.0004,
+                    "BottomMeasuredDepth": 2481.0,
+                    "Curves": [
+                        {
+                            "Mnemonic": "DEPT",
+                            "TopDepth": 2182.0,
+                            "BaseDepth": 2481.0,
+                            "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                            "CurveUnit": "osdu:reference-data--UnitOfMeasure:M:"
+                        },
+                        {
+                            "Mnemonic": "GR",
+                            "TopDepth": 2182.0,
+                            "BaseDepth": 2481.0,
+                            "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                            "CurveUnit": "osdu:reference-data--UnitOfMeasure:GAPI:"
+                        },
+                        {
+                            "Mnemonic": "DT",
+                            "TopDepth": 2182.0,
+                            "BaseDepth": 2481.0,
+                            "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                            "CurveUnit": "osdu:reference-data--UnitOfMeasure:US/F:"
+                        },
+                        {
+                            "Mnemonic": "RHOB",
+                            "TopDepth": 2182.0,
+                            "BaseDepth": 2481.0,
+                            "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                            "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:"
+                        },
+                        {
+                            "Mnemonic": "DRHO",
+                            "TopDepth": 2182.0,
+                            "BaseDepth": 2481.0,
+                            "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                            "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:"
+                        },
+                        {
+                            "Mnemonic": "NPHI",
+                            "TopDepth": 2182.0,
+                            "BaseDepth": 2481.0,
+                            "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                            "CurveUnit": "osdu:reference-data--UnitOfMeasure:V/V:"
+                        }
+                    ]
+                }
+            }
+        ],
+        "Datasets": [
+            {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:LAS 2:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las"
+                        }
+                    }
+                }
+            },
+            {
+                "id": "surrogate-key:file-2",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:LAS 2:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las"
+                        }
+                    }
+                }
+            }
+        ]
+    }
+}
diff --git a/tests/plugin-unit-tests/data/workProduct/ref_result_r3_Welllog.json b/tests/plugin-unit-tests/data/workProduct/ref_result_r3_Welllog.json
new file mode 100644
index 0000000000000000000000000000000000000000..5c09bbcfeadc404eec05e019d9906da01ab83767
--- /dev/null
+++ b/tests/plugin-unit-tests/data/workProduct/ref_result_r3_Welllog.json
@@ -0,0 +1,9 @@
+[
+    "osdu:reference-data--ResourceSecurityClassification:RESTRICTED",
+    "osdu:master-data--Wellbore:1013",
+    "osdu:reference-data--UnitOfMeasure:M",
+    "osdu:reference-data--UnitOfMeasure:GAPI",
+    "osdu:reference-data--UnitOfMeasure:US/F",
+    "osdu:reference-data--UnitOfMeasure:G/C3",
+    "osdu:reference-data--UnitOfMeasure:V/V"
+]
diff --git a/tests/plugin-unit-tests/data/workProduct/traversal_r3_Welllog.json b/tests/plugin-unit-tests/data/workProduct/traversal_r3_Welllog.json
new file mode 100644
index 0000000000000000000000000000000000000000..55eba5b875dfe0bd25066020137177c8fa7afa74
--- /dev/null
+++ b/tests/plugin-unit-tests/data/workProduct/traversal_r3_Welllog.json
@@ -0,0 +1,120 @@
+[
+    {
+        "entity": {
+            "kind": "osdu:wks:work-product--WorkProduct:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "AKM-11 LOG",
+                "Description": "Well Log",
+                "Components": [
+                    "surrogate-key:wpc-1"
+                ]
+            }
+        },
+        "schema": "",
+        "type": "wpc"
+    },
+    {
+        "entity": {
+            "id": "surrogate-key:wpc-1",
+            "kind": "osdu:wks:work-product-component--WellLog:1.0.0",
+            "acl": {
+                "owners": [],
+                "viewers": []
+            },
+            "legal": {
+                "legaltags": [],
+                "otherRelevantDataCountries": []
+            },
+            "data": {
+                "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                "Name": "AKM-11 LOG",
+                "Description": "Well Log",
+                "Datasets": [
+                    "surrogate-key:file-1"
+                ],
+                "WellboreID": "osdu:master-data--Wellbore:1013:",
+                "TopMeasuredDepth": 2182.0004,
+                "BottomMeasuredDepth": 2481.0,
+                "Curves": [
+                    {
+                        "Mnemonic": "DEPT",
+                        "TopDepth": 2182.0,
+                        "BaseDepth": 2481.0,
+                        "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                        "CurveUnit": "osdu:reference-data--UnitOfMeasure:M:"
+                    },
+                    {
+                        "Mnemonic": "GR",
+                        "TopDepth": 2182.0,
+                        "BaseDepth": 2481.0,
+                        "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                        "CurveUnit": "osdu:reference-data--UnitOfMeasure:GAPI:"
+                    },
+                    {
+                        "Mnemonic": "DT",
+                        "TopDepth": 2182.0,
+                        "BaseDepth": 2481.0,
+                        "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                        "CurveUnit": "osdu:reference-data--UnitOfMeasure:US/F:"
+                    },
+                    {
+                        "Mnemonic": "RHOB",
+                        "TopDepth": 2182.0,
+                        "BaseDepth": 2481.0,
+                        "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                        "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:"
+                    },
+                    {
+                        "Mnemonic": "DRHO",
+                        "TopDepth": 2182.0,
+                        "BaseDepth": 2481.0,
+                        "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                        "CurveUnit": "osdu:reference-data--UnitOfMeasure:G/C3:"
+                    },
+                    {
+                        "Mnemonic": "NPHI",
+                        "TopDepth": 2182.0,
+                        "BaseDepth": 2481.0,
+                        "DepthUnit": "osdu:reference-data--UnitOfMeasure:M:",
+                        "CurveUnit": "osdu:reference-data--UnitOfMeasure:V/V:"
+                    }
+                ]
+            }
+        },
+        "schema": ""
+    },
+    {
+        "entity": {
+                "id": "surrogate-key:file-1",
+                "kind": "osdu:wks:dataset--File.Generic:1.0.0",
+                "acl": {
+                    "owners": [],
+                    "viewers": []
+                },
+                "legal": {
+                    "legaltags": [],
+                    "otherRelevantDataCountries": []
+                },
+                "data": {
+                    "ResourceSecurityClassification": "osdu:reference-data--ResourceSecurityClassification:RESTRICTED:",
+                    "SchemaFormatTypeID": "osdu:reference-data--SchemaFormatType:LAS 2:",
+                    "DatasetProperties": {
+                        "FileSourceInfo": {
+                            "FileSource": "",
+                            "PreloadFilePath": "s3://osdu-seismic-test-data/r1/data/provided/well-logs/1013_akm11_1978_comp.las"
+                        }
+                    }
+                }
+            },
+        "schema": ""
+    }
+]
diff --git a/tests/plugin-unit-tests/file_paths.py b/tests/plugin-unit-tests/file_paths.py
index e2368312f0c3a9f31beea5285c927444dd4ccdab..796a39693101ad2255c5a44a2c97ef72a8072c25 100644
--- a/tests/plugin-unit-tests/file_paths.py
+++ b/tests/plugin-unit-tests/file_paths.py
@@ -18,9 +18,11 @@ import os
 DATA_PATH_PREFIX = f"{os.path.dirname(__file__)}/data"
 
 MANIFEST_GENERIC_SCHEMA_PATH = f"{DATA_PATH_PREFIX}/manifests/schema_Manifest.1.0.0.json"
+MANIFEST_NEW_GENERIC_SCHEMA_PATH = f"{DATA_PATH_PREFIX}/manifests/new_schema_Manifest.1.0.0.json"
 MANIFEST_GENERIC_PATH = f"{DATA_PATH_PREFIX}/manifests/Manifest.1.0.0.json"
 
 MANIFEST_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/Wellbore.0.3.0.json"
+MANIFEST_BATCH_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/batch_Wellbore.0.3.0.json"
 SCHEMA_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/schema_Wellbore.3.0.0.json"
 RECORD_WELLBORE_VALID_PATH = f"{DATA_PATH_PREFIX}/master/record_Wellbore.0.3.0.json"
 SCHEMA_GENERIC_MASTERDATA_PATH = f"{DATA_PATH_PREFIX}/master/schema_GenericMasterData.1.0.0.json"
@@ -34,8 +36,38 @@ SCHEMA_WORK_PRODUCT_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_WorkPro
 SCHEMA_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/schema_SeismicTraceData.1.0.0.json"
 RECORD_SEISMIC_TRACE_DATA_VALID_PATH = f"{DATA_PATH_PREFIX}/workProduct/record_SeismicTraceData.json"
 
+SURROGATE_MANIFEST_WELLBORE = f"{DATA_PATH_PREFIX}/surrogate/Wellbore.0.3.0.json"
+
 MANIFEST_EMPTY_PATH = f"{DATA_PATH_PREFIX}/invalid/EmptyManifest.json"
 TRAVERSAL_MANIFEST_EMPTY_PATH = f"{DATA_PATH_PREFIX}/invalid/TraversalEmptyManifest.json"
 
 SEARCH_VALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseValid.json"
 SEARCH_INVALID_RESPONSE_PATH = f"{DATA_PATH_PREFIX}/other/SearchResponseInvalid.json"
+SEARCH_EXTRACTED_IDS_PATH = f"{DATA_PATH_PREFIX}/other/ExtractedIds.json"
+
+
+SURROGATE_MANIFEST_WELLBORE = f"{DATA_PATH_PREFIX}/surrogate/Wellbore.0.3.0.json"
+
+MANIFEST_WELL_PATH = f"{DATA_PATH_PREFIX}/master/r3_Well.json"
+TRAVERSAL_WELL_PATH = f"{DATA_PATH_PREFIX}/master/traversal_r3_Well.json"
+REF_RESULT_WELL_PATH = f"{DATA_PATH_PREFIX}/master/ref_result_r3_Well.json"
+
+
+MANIFEST_WELLLOG_PATH = f"{DATA_PATH_PREFIX}/workProduct/r3_Welllog.json"
+TRAVERSAL_WELLLOG_PATH = f"{DATA_PATH_PREFIX}/workProduct/traversal_r3_Welllog.json"
+REF_RESULT_WELLLOG_PATH = f"{DATA_PATH_PREFIX}/workProduct/ref_result_r3_Welllog.json"
+BATCH_MANIFEST_WELLBORE = f"{DATA_PATH_PREFIX}/batchManifest/Wellbore.0.3.0.json"
+
+DATA_INTEGRITY_VALID_DATA = f"{DATA_PATH_PREFIX}/data_integrity/valid_data.json"
+DATA_INTEGRITY_ORPHAN_DATASETS = f"{DATA_PATH_PREFIX}/data_integrity/orphan_datasets.json"
+DATA_INTEGRITY_VALID_WP_INVALID_WPC = f"{DATA_PATH_PREFIX}/data_integrity/valid_wp_invalid_wpc.json"
+DATA_INTEGRITY_INVALID_WP = f"{DATA_PATH_PREFIX}/data_integrity/invalid_wp.json"
+DATA_INTEGRITY_EMPTY_DATA = f"{DATA_PATH_PREFIX}/data_integrity/empty_data.json"
+DATA_INTEGRITY_EMPTY_DATA_CASE_2 = f"{DATA_PATH_PREFIX}/data_integrity/empty_data_inside.json"
+DATA_INTEGRITY_EMPTY_WP = f"{DATA_PATH_PREFIX}/data_integrity/empty_wp.json"
+DATA_INTEGRITY_VALID_REAL_IDS = f"{DATA_PATH_PREFIX}/data_integrity/valid_data_real_ids.json"
+
+FILES_SOURCE_VALID = f"{DATA_PATH_PREFIX}/data_integrity/file_source/valid_files.json"
+FILES_SOURCE_INVALID = f"{DATA_PATH_PREFIX}/data_integrity/file_source/invalid_files.json"
+FILE_COLLECTIONS_VALID = f"{DATA_PATH_PREFIX}/data_integrity/file_source/valid_file_collections.json"
+FILE_COLLECTIONS_INVALID = f"{DATA_PATH_PREFIX}/data_integrity/file_source/invalid_file_collections.json"
diff --git a/tests/plugin-unit-tests/test_data_integrity_validator.py b/tests/plugin-unit-tests/test_data_integrity_validator.py
new file mode 100644
index 0000000000000000000000000000000000000000..b48d35fe315680cb2cf7db1cbe08e2295bd704d0
--- /dev/null
+++ b/tests/plugin-unit-tests/test_data_integrity_validator.py
@@ -0,0 +1,144 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+import json
+import os
+import sys
+
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins")
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
+
+import pytest
+import pytest_mock
+
+from file_paths import (DATA_INTEGRITY_VALID_DATA, DATA_INTEGRITY_ORPHAN_DATASETS,
+                        DATA_INTEGRITY_VALID_WP_INVALID_WPC, DATA_INTEGRITY_INVALID_WP,
+                        DATA_INTEGRITY_EMPTY_DATA, DATA_INTEGRITY_VALID_REAL_IDS,
+                        DATA_INTEGRITY_EMPTY_DATA_CASE_2, DATA_INTEGRITY_EMPTY_WP)
+from libs.search_client import SearchClient, SearchResponse
+from libs.validation.validate_data_integrity import DataIntegrityValidator
+from libs.validation.validate_file_source import FileSourceValidator
+
+
+class TestDataIntegrityValidator:
+    """Test data integrity validator."""
+    @pytest.fixture
+    def provide_manifests(self, expected_manifest_path, input_manifest_path):
+        """Read manifest from files."""
+        with open(expected_manifest_path) as f:
+            expected_manifest = json.load(f)
+        with open(input_manifest_path) as f:
+            input_manifest = json.load(f)
+        return expected_manifest, input_manifest
+
+    @pytest.fixture
+    def file_source_validator(self):
+        file_source_validator = FileSourceValidator()
+        return file_source_validator
+
+    @pytest.mark.parametrize("expected_manifest_path, input_manifest_path", [
+        pytest.param(DATA_INTEGRITY_VALID_DATA, DATA_INTEGRITY_VALID_DATA),
+        pytest.param(DATA_INTEGRITY_VALID_DATA, DATA_INTEGRITY_ORPHAN_DATASETS),
+        pytest.param(DATA_INTEGRITY_VALID_DATA, DATA_INTEGRITY_VALID_WP_INVALID_WPC),
+        pytest.param(DATA_INTEGRITY_EMPTY_DATA, DATA_INTEGRITY_INVALID_WP),
+    ])
+    def test_validate_data_integrity(self, mocker: pytest_mock.MockerFixture, provide_manifests,
+                                     file_source_validator, expected_manifest_path: str,
+                                     input_manifest_path: str):
+        """Test validation of datasets dependencies."""
+        search_client = mocker.Mock(spec=SearchClient)
+        data_integrity_validator = DataIntegrityValidator(search_client, file_source_validator)
+
+        expected_manifest, input_manifest = provide_manifests
+
+        data_integrity_validator.validate_manifest_data_integrity(input_manifest)
+
+        assert expected_manifest == input_manifest
+
+    @pytest.mark.parametrize("expected_manifest_path, input_manifest_path", [
+        pytest.param(DATA_INTEGRITY_EMPTY_DATA, DATA_INTEGRITY_EMPTY_DATA),
+        pytest.param(DATA_INTEGRITY_EMPTY_DATA, DATA_INTEGRITY_EMPTY_DATA_CASE_2),
+        pytest.param(DATA_INTEGRITY_EMPTY_DATA, DATA_INTEGRITY_EMPTY_WP)
+    ])
+    def test_validate_empty_data_integrity(self, mocker: pytest_mock.MockerFixture,
+                                           provide_manifests, file_source_validator,
+                                           expected_manifest_path: str, input_manifest_path: str):
+        """Test validation of datasets dependencies."""
+        search_client = mocker.Mock(spec=SearchClient)
+        data_integrity_validator = DataIntegrityValidator(search_client, file_source_validator)
+
+        expected_manifest, input_manifest = provide_manifests
+
+        data_integrity_validator.validate_manifest_data_integrity(input_manifest)
+
+        assert expected_manifest == input_manifest
+
+    @pytest.mark.parametrize("expected_manifest_path, input_manifest_path, wpc_ids, datasets_ids", [
+        pytest.param(DATA_INTEGRITY_VALID_REAL_IDS, DATA_INTEGRITY_VALID_REAL_IDS,
+                     ["opendes:work-product-component--GenericWorkProductComponent:1234"],
+                     ["opendes:dataset--GenericDataset:1234"]),
+        pytest.param(DATA_INTEGRITY_EMPTY_DATA, DATA_INTEGRITY_VALID_REAL_IDS, [], []),
+    ])
+    def test_validate_data_integrity_real_ids(self, mocker: pytest_mock.MockerFixture,
+                                              provide_manifests, file_source_validator,
+                                              expected_manifest_path: str, input_manifest_path: str,
+                                              wpc_ids: list, datasets_ids: list):
+        """Test validation of datasets dependencies simulating search api call."""
+        search_client = mocker.Mock(spec=SearchClient)
+        data_integrity_validator = DataIntegrityValidator(search_client, file_source_validator)
+        mocker.patch(
+            "libs.validation.validate_data_integrity.DataIntegrityValidator._search_for_entities",
+            side_effect=[datasets_ids, wpc_ids])
+        expected_manifest, input_manifest = provide_manifests
+
+        data_integrity_validator.validate_manifest_data_integrity(input_manifest)
+
+        assert expected_manifest == input_manifest
+
+    @pytest.mark.parametrize("expected_manifest_path, input_manifest_path, wpc_ids, datasets_ids", [
+        pytest.param(DATA_INTEGRITY_VALID_REAL_IDS, DATA_INTEGRITY_VALID_REAL_IDS,
+                     [{
+                         "id": "opendes:work-product-component--GenericWorkProductComponent:1234"
+                     }], [{
+                         "id": "opendes:dataset--GenericDataset:1234"
+                     }])
+    ])
+    def test_validate_expected_search_calls(self, mocker: pytest_mock.MockerFixture,
+                                            provide_manifests, file_source_validator,
+                                            expected_manifest_path: str, input_manifest_path: str,
+                                            wpc_ids: list, datasets_ids: list):
+        """Test validation of datasets dependencies simulating search api call."""
+        datasets_response = SearchResponse(results=datasets_ids, total_count=1)
+        wpcs_response = SearchResponse(results=wpc_ids, total_count=1)
+
+        search_client = mocker.Mock(spec=SearchClient)
+        query_records_mock = mocker.Mock(side_effect=[datasets_response, wpcs_response])
+        search_client.query_records = query_records_mock
+        data_integrity_validator = DataIntegrityValidator(search_client, file_source_validator)
+        expected_manifest, input_manifest = provide_manifests
+
+        data_integrity_validator.validate_manifest_data_integrity(input_manifest)
+
+        assert expected_manifest == input_manifest
+        datasets_call = mocker.call(kind='*:*:*:*',
+                                    query_str='id:("opendes:dataset--GenericDataset:1234")',
+                                    limit=1,
+                                    returned_fields=['id'])
+        wpcs_call = mocker.call(
+            kind='*:*:*:*',
+            query_str='id:("opendes:work-product-component--GenericWorkProductComponent:1234")',
+            limit=1,
+            returned_fields=['id'])
+        query_records_mock.assert_has_calls([datasets_call, wpcs_call])
diff --git a/tests/plugin-unit-tests/test_file_handler.py b/tests/plugin-unit-tests/test_file_handler.py
index 8e5ba9ae22262ce2f0fff455043dad59bf49ec75..742d70637e480ca31fcbc4d1c356bb902481224f 100644
--- a/tests/plugin-unit-tests/test_file_handler.py
+++ b/tests/plugin-unit-tests/test_file_handler.py
@@ -38,13 +38,13 @@ from libs.handle_file import FileDownloadUrlResponse, FileUploadUrlResponse, Fil
 
 class TestFileHandler:
 
-    BASE_TEST_URL = "http://file_service_url"
+    BASE_TEST_HOST = "http://file_service_url"
     PARTITION_ID = "test_partition_id"
 
     @pytest.fixture()
     def file_handler(self, monkeypatch):
         context = Context(data_partition_id=self.PARTITION_ID, app_key="")
-        file_handler = FileHandler(self.BASE_TEST_URL,
+        file_handler = FileHandler(self.BASE_TEST_HOST,
                                    AirflowTokenRefresher(get_test_credentials()),
                                    context)
         monkeypatch.setattr(
@@ -61,7 +61,7 @@ class TestFileHandler:
     @responses.activate
     def test_get_file_staging_location(self, file_handler: FileHandler):
         test_staging_location = "gs://staging/test/file_id"
-        responses.add(responses.POST, f"{self.BASE_TEST_URL}/getFileLocation",
+        responses.add(responses.POST, f"{self.BASE_TEST_HOST}/getFileLocation",
                       json={"Location": test_staging_location}, status=http.HTTPStatus.OK)
 
         assert test_staging_location == file_handler.get_file_staging_location("/test/file_id")
@@ -76,7 +76,7 @@ class TestFileHandler:
     @responses.activate
     def test_get_file_staging_location_error(self, file_handler: FileHandler,
                                                  http_status: str, reason: str):
-        responses.add(responses.POST, f"{self.BASE_TEST_URL}/getFileLocation",
+        responses.add(responses.POST, f"{self.BASE_TEST_HOST}/getFileLocation",
                       status=http_status, body=reason)
 
         with pytest.raises((tenacity.RetryError, requests.exceptions.HTTPError)):
@@ -91,7 +91,7 @@ class TestFileHandler:
             "unsignedUrl": test_permanent_location,
             "kind": "test_kind"
         }
-        responses.add(responses.GET, f"{self.BASE_TEST_URL}/v1/files/{test_record_id}/downloadURL",
+        responses.add(responses.GET, f"{self.BASE_TEST_HOST}/v1/files/{test_record_id}/downloadURL",
                       json=json_response, status=http.HTTPStatus.OK)
 
         assert test_permanent_location == file_handler.get_file_permanent_location(test_record_id)
@@ -106,7 +106,7 @@ class TestFileHandler:
     def test_get_file_permanent_location_error(self, file_handler: FileHandler,
                                                http_status: str, reason: str):
         test_record_id = "test_record_id"
-        responses.add(responses.GET, f"{self.BASE_TEST_URL}/v1/files/{test_record_id}/downloadURL",
+        responses.add(responses.GET, f"{self.BASE_TEST_HOST}/v1/files/{test_record_id}/downloadURL",
                       status=http_status, body=reason)
 
         with pytest.raises((tenacity.RetryError, requests.exceptions.HTTPError)):
@@ -124,7 +124,7 @@ class TestFileHandler:
             file_record, unused_wpc_record, unused_wp_record = json.load(cf)
 
         test_record_id = "test_record_id"
-        responses.add(responses.POST, f"{self.BASE_TEST_URL}/v1/files/metadata",
+        responses.add(responses.POST, f"{self.BASE_TEST_HOST}/v1/files/metadata",
                       json={"id": test_record_id}, status=http.HTTPStatus.OK)
 
         assert test_record_id == file_handler.save_file_record(file_record)
diff --git a/tests/plugin-unit-tests/test_file_source_validator.py b/tests/plugin-unit-tests/test_file_source_validator.py
new file mode 100644
index 0000000000000000000000000000000000000000..43acc65dcb553e0e26a3b64d860ff8dadb5aaca5
--- /dev/null
+++ b/tests/plugin-unit-tests/test_file_source_validator.py
@@ -0,0 +1,65 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+import json
+import os
+import sys
+
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins")
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
+
+import pytest
+import pytest_mock
+
+from file_paths import (FILES_SOURCE_VALID, FILES_SOURCE_INVALID, FILE_COLLECTIONS_VALID,
+                        FILE_COLLECTIONS_INVALID)
+from libs.validation.validate_file_source import FileSourceValidator
+
+
+class TestFileSourceValidator:
+    """Test validation of file source in datasets File and FileCollection."""
+    @pytest.fixture
+    def provide_datasets(self, input_datasets_path) -> list:
+        """Read data from files."""
+        with open(input_datasets_path) as f:
+            return json.load(f)
+
+    @pytest.mark.parametrize("input_datasets_path", [
+        pytest.param(FILES_SOURCE_VALID),
+        pytest.param(FILE_COLLECTIONS_VALID),
+    ])
+    def test_valid_datasets(self, provide_datasets: list, input_datasets_path: str):
+        """Test valid inputs."""
+        file_source_validator = FileSourceValidator()
+        expected_datasets = sorted(provide_datasets, key=lambda k: k["id"])
+
+        filtered_datasets = file_source_validator.filter_valid_datasets(provide_datasets)
+        filtered_datasets = sorted(filtered_datasets, key=lambda k: k["id"])
+
+        assert len(expected_datasets) == len(filtered_datasets)
+        for i in range(len(expected_datasets)):
+            assert expected_datasets[i] == filtered_datasets[i]
+
+    @pytest.mark.parametrize("input_datasets_path", [
+        pytest.param(FILES_SOURCE_INVALID),
+        pytest.param(FILE_COLLECTIONS_INVALID),
+    ])
+    def test_invalid_datasets(self, provide_datasets: list, input_datasets_path: str):
+        file_source_validator = FileSourceValidator()
+
+        expected_datasets = []
+        filtered_datasets = file_source_validator.filter_valid_datasets(provide_datasets)
+
+        assert expected_datasets == filtered_datasets
diff --git a/tests/plugin-unit-tests/test_manifest_analyzer.py b/tests/plugin-unit-tests/test_manifest_analyzer.py
new file mode 100644
index 0000000000000000000000000000000000000000..a93a278b044c698542f0a5c309f2416f7bb481fd
--- /dev/null
+++ b/tests/plugin-unit-tests/test_manifest_analyzer.py
@@ -0,0 +1,201 @@
+#  Copyright 2020 Google LLC
+#  Copyright 2020 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+
+import json
+import os
+import logging
+import sys
+
+from functools import partial
+
+import pytest
+from file_paths import SURROGATE_MANIFEST_WELLBORE
+
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins")
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
+
+from libs.manifest_analyzer import ManifestAnalyzer, EntityNode
+from libs.traverse_manifest import ManifestEntity
+from libs.refresh_token import AirflowTokenRefresher
+from libs.context import Context
+
+logger = logging.getLogger()
+
+TEST_FAKE_DATA = [
+        {
+            "id": "1",
+            "parents": [],
+        },
+        {
+            "id": "2",
+            "parents": ["1"],
+        },
+        {
+            "id": "3",
+            "parents": ["1"],
+        },
+        {
+            "id": "4",
+            "parents": ["2"],
+        },
+        {
+            "id": "5",
+            "parents": ["1", "3"],
+        },
+        {
+            "id": "7",
+            "parents": ["1", "6"],
+        },
+        {
+            "id": "9",
+            "parents": ["7"]
+        }
+    ]
+
+TEST_FAKE_DATA = [ManifestEntity(entity=i, schema="") for i in TEST_FAKE_DATA]
+
+class TestManifestAnalyzer(object):
+
+    @pytest.fixture()
+    def manifest_analyzer(self):
+        with open(SURROGATE_MANIFEST_WELLBORE) as f:
+            data = json.load(f)
+        context = Context(data_partition_id="test", app_key="test")
+        token_refresher = AirflowTokenRefresher()
+        return ManifestAnalyzer(data, "http://test.com", token_refresher, context)
+
+    @pytest.fixture()
+    def fake_data_manifest_analyzer(self, monkeypatch, data, is_in_storage: bool = False):
+        context = Context(data_partition_id="test", app_key="test")
+        token_refresher = AirflowTokenRefresher()
+        monkeypatch.setattr(EntityNode, "get_parent_srns", self.mock_get_parent_srns)
+        monkeypatch.setattr(ManifestAnalyzer, "_is_in_storage", lambda self, srn: is_in_storage)
+        manifest_analyzer = ManifestAnalyzer(data, "http://test", token_refresher, context)
+        return manifest_analyzer
+
+    def process_entity(self, entity: EntityNode) -> str:
+        return f"system_srn: {entity.srn}"
+
+    def index_in_queue_by_srn(
+        self,
+        manifest_analyzer: ManifestAnalyzer,
+        queue: list,
+        srn: str
+    ):
+        entity_node = manifest_analyzer.srn_node_table[srn]
+        return queue.index(entity_node)
+
+    @staticmethod
+    def mock_get_parent_srns(obj: EntityNode):
+        parent_srns = set(obj.content.get("parents", []))
+        return parent_srns
+
+    @pytest.mark.parametrize(
+        "data",
+        [
+            pytest.param(TEST_FAKE_DATA, id="Fake data")
+        ]
+    )
+    def test_queue_order(
+        self,
+        monkeypatch,
+        fake_data_manifest_analyzer: ManifestAnalyzer,
+        data: dict
+    ):
+        """
+        Here we use array with simple objects where it's immediately seen who depends on whom.
+        Check if queue return parents, then and only then their children.
+        Check if there is no orphaned and their children in the queue (SRN 7 and SRN 9).
+        """
+        queue = list(fake_data_manifest_analyzer.entity_queue())
+        index_in_queue = partial(self.index_in_queue_by_srn, fake_data_manifest_analyzer, queue)
+
+        # check if child goes after all its parents in queue.
+        assert index_in_queue("5") > index_in_queue("1") \
+               and index_in_queue("5") > index_in_queue("3"), \
+            "SRN 5 must follow parents: SRN 1 and 3"
+
+        # check if orphans and their dependants are not in ingestion queue.
+        for unprocessed_srn in ("7", "9"):
+            unprocessed_entity = fake_data_manifest_analyzer.srn_node_table[unprocessed_srn]
+            assert unprocessed_entity not in queue, \
+                f"{unprocessed_entity} expected not to be in queue: {queue}"
+
+    @pytest.mark.parametrize(
+        "data",
+        [
+            pytest.param(TEST_FAKE_DATA, id="Fake data")
+        ]
+    )
+    def test_add_new_unporcessed(
+        self,
+        monkeypatch,
+        fake_data_manifest_analyzer: ManifestAnalyzer,
+        data: dict
+    ):
+        """
+        Here we use array with simple objects where it's immediately seen who depends on whom.
+        Imagine we can't process entity (e.g. Storage service can't save this entity).
+        Then we must add this entity to unprocessed ones and traverse all the children of
+        this entity marking them as unprocessed.
+        They must disappear from the ingestion queue.
+        """
+
+        queue = fake_data_manifest_analyzer.entity_queue()
+        unprocessed_entity = fake_data_manifest_analyzer.srn_node_table["3"]
+        expected_unprocessed_entities = {"7", "9", "3", "5"}
+        fake_data_manifest_analyzer.add_unprocessed_entity(unprocessed_entity)
+        for entity in queue:
+            assert entity not in expected_unprocessed_entities, \
+                f"{entity} must be excluded from queue."
+
+    @pytest.mark.parametrize(
+        "data, is_in_storage",
+        [
+            pytest.param(TEST_FAKE_DATA, True, id="Fake data")
+        ]
+    )
+    def test_all_missed_parents_are_in_storage(
+        self,
+        monkeypatch,
+        data: dict,
+        is_in_storage: bool
+    ):
+        context = Context(data_partition_id="test", app_key="test")
+        token_refresher = AirflowTokenRefresher()
+        monkeypatch.setattr(EntityNode, "get_parent_srns", self.mock_get_parent_srns)
+        monkeypatch.setattr(ManifestAnalyzer, "_is_in_storage", lambda self, srn: True)
+        manifest_analyzer = ManifestAnalyzer(data, "http://test", token_refresher, context)
+        queue = list(manifest_analyzer.entity_queue())
+        assert not manifest_analyzer.unprocessed_entities, \
+            "If absent parents are in storage, there are no orphaned child then."
+
+    def test_real_data(self):
+        with open(SURROGATE_MANIFEST_WELLBORE) as f:
+            data = json.load(f)
+        data = [ManifestEntity(entity=e, schema="") for e in data]
+        context = Context(data_partition_id="test", app_key="test")
+        token_refresher = AirflowTokenRefresher()
+        manifest_analyzer = ManifestAnalyzer(data, "http://test.com", token_refresher, context)
+
+        for entity in manifest_analyzer.entity_queue():
+            entity.replace_parents_surrogate_srns()
+            entity.system_srn = self.process_entity(entity)
+            logger.info(f"Processed entity {json.dumps(entity.content, indent=2)}")
+
+        logger.info("\n")
+        logger.info(f"Processed entities: {manifest_analyzer.processed_entities}")
+        logger.info(f"Unprocessed entities {manifest_analyzer.unprocessed_entities}")
diff --git a/tests/plugin-unit-tests/test_manifest_integrity.py b/tests/plugin-unit-tests/test_manifest_integrity.py
new file mode 100644
index 0000000000000000000000000000000000000000..ab67c21816092764fa872500053ed53936df2143
--- /dev/null
+++ b/tests/plugin-unit-tests/test_manifest_integrity.py
@@ -0,0 +1,72 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+import json
+import os
+import sys
+
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins")
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
+
+import pytest
+from mock_providers import get_test_credentials
+from file_paths import MANIFEST_WELL_PATH, REF_RESULT_WELL_PATH, MANIFEST_WELLLOG_PATH, REF_RESULT_WELLLOG_PATH
+from libs.refresh_token import AirflowTokenRefresher
+from libs.validation.validate_referential_integrity import ManifestIntegrity
+
+
+class TestIntegrityProvider:
+
+    @pytest.fixture
+    def manifest_integrity(self, monkeypatch, conf_path: str):
+        with open(conf_path) as f:
+            conf = json.load(f)
+        manifest_integrity = ManifestIntegrity("", AirflowTokenRefresher(get_test_credentials()), conf)
+        return manifest_integrity
+
+    @pytest.mark.parametrize(
+        "conf_path,ref_result_file",
+        [
+            pytest.param(
+                MANIFEST_WELL_PATH,
+                REF_RESULT_WELL_PATH,
+                id="Validate manifest Master")
+        ]
+    )
+    def test_extract_references_master(self, monkeypatch, manifest_integrity, conf_path: str, ref_result_file: str):
+        with open(ref_result_file) as f:
+            expected_result = json.load(f)
+
+        manifest_integrity._collect_ids(manifest_integrity.context)
+        manifest_records = manifest_integrity._extract_references(
+            manifest_integrity.context["MasterData"][0])
+        assert set(manifest_records) == set(expected_result)
+
+    @pytest.mark.parametrize(
+        "conf_path,ref_result_file",
+        [
+            pytest.param(
+                MANIFEST_WELLLOG_PATH,
+                REF_RESULT_WELLLOG_PATH,
+                id="Validate manifest WPC")
+        ]
+    )
+    def test_extract_references_wpc(self, monkeypatch, manifest_integrity, conf_path: str, ref_result_file: str):
+        with open(ref_result_file) as f:
+            expected_result = json.load(f)
+        manifest_integrity._collect_ids(manifest_integrity.context)
+        manifest_records = manifest_integrity._extract_references(
+            manifest_integrity.context["Data"]["WorkProductComponents"][0])
+        assert set(manifest_records) == set(expected_result)
diff --git a/tests/plugin-unit-tests/test_manifest_processor_r3.py b/tests/plugin-unit-tests/test_manifest_processor_r3.py
index 8fbab18ea28a5925f7edbd1b2590b69ed245617b..c44f7835a22472a8a6108c40067ee40bee480954 100644
--- a/tests/plugin-unit-tests/test_manifest_processor_r3.py
+++ b/tests/plugin-unit-tests/test_manifest_processor_r3.py
@@ -26,8 +26,8 @@ from libs.context import Context
 from libs.handle_file import FileHandler
 from libs.source_file_check import SourceFileChecker
 from libs.refresh_token import AirflowTokenRefresher
+from libs.traverse_manifest import ManifestEntity
 from libs.exceptions import EmptyManifestError
-from deepdiff import DeepDiff
 import pytest
 import requests
 
@@ -77,8 +77,14 @@ class TestManifestProcessor:
 
         monkeypatch.setattr(requests, "put", mockresponse)
 
+    @pytest.fixture()
+    def manifest_records(self, traversal_manifest_file: str) -> list:
+        with open(traversal_manifest_file) as f:
+            manifest_file = json.load(f)
+        return manifest_file
+
     @pytest.fixture(autouse=True)
-    def manifest_processor(self, monkeypatch, conf_path: str, traversal_manifest_file: str):
+    def manifest_processor(self, monkeypatch, traversal_manifest_file, conf_path: str):
         with open(conf_path) as f:
             conf = json.load(f)
         with open(traversal_manifest_file) as f:
@@ -90,7 +96,6 @@ class TestManifestProcessor:
         source_file_checker = SourceFileChecker()
         manifest_processor = process_manifest_r3.ManifestProcessor(
             storage_url="",
-            manifest_records=manifest_records,
             token_refresher=token_refresher,
             context=context,
             file_handler=file_handler,
@@ -132,6 +137,7 @@ class TestManifestProcessor:
         self,
         monkeypatch,
         manifest_processor: process_manifest_r3.ManifestProcessor,
+        manifest_records,
         mock_records_list: list,
         traversal_manifest_file: str,
         conf_path: str,
@@ -150,6 +156,7 @@ class TestManifestProcessor:
     def test_save_record_invalid_storage_response_value(
         self,
         monkeypatch,
+        manifest_records,
         manifest_processor: process_manifest_r3.ManifestProcessor,
         traversal_manifest_file: str,
         conf_path: str
@@ -169,6 +176,7 @@ class TestManifestProcessor:
     def test_save_record_storage_response_http_error(
         self,
         monkeypatch,
+        manifest_records,
         manifest_processor: process_manifest_r3.ManifestProcessor,
         traversal_manifest_file: str,
         conf_path: str
@@ -191,12 +199,14 @@ class TestManifestProcessor:
     def test_process_manifest_valid(
         self,
         monkeypatch,
+        manifest_records,
         manifest_processor: process_manifest_r3.ManifestProcessor,
         traversal_manifest_file: str,
         conf_path: str
     ):
         self.monkeypatch_storage_response(monkeypatch)
-        manifest_processor.process_manifest()
+        manifest_file = [ManifestEntity(entity=e["entity"], schema=e["schema"]) for e in manifest_records]
+        manifest_processor.process_manifest_records(manifest_file)
 
     @pytest.mark.parametrize(
         "conf_path,traversal_manifest_file",
@@ -207,13 +217,14 @@ class TestManifestProcessor:
     def test_process_empty_manifest(
         self,
         monkeypatch,
+        manifest_records,
         manifest_processor: process_manifest_r3.ManifestProcessor,
         traversal_manifest_file: str,
         conf_path: str
     ):
         self.monkeypatch_storage_response(monkeypatch)
         with pytest.raises(EmptyManifestError):
-            manifest_processor.process_manifest()
+            manifest_processor.process_manifest_records(manifest_records)
 
     @pytest.mark.parametrize(
         "conf_path,expected_kind_name,traversal_manifest_file",
@@ -225,10 +236,11 @@ class TestManifestProcessor:
         self,
         monkeypatch,
         manifest_processor: process_manifest_r3.ManifestProcessor,
+        manifest_records: list,
         conf_path: str,
         traversal_manifest_file: str,
         expected_kind_name: str
     ):
-        for manifest_part in manifest_processor.manifest_records:
+        for manifest_part in manifest_records:
             kind = manifest_part["entity"]["kind"]
             assert expected_kind_name == manifest_processor._get_kind_name(kind)
diff --git a/tests/plugin-unit-tests/test_manifest_traversal.py b/tests/plugin-unit-tests/test_manifest_traversal.py
index 2b0c392cff5f32b4838a1d123a3716965f8b16e5..38dcd75e7eef6c0b4d4227d2a2d8670d35d7d819 100644
--- a/tests/plugin-unit-tests/test_manifest_traversal.py
+++ b/tests/plugin-unit-tests/test_manifest_traversal.py
@@ -25,7 +25,7 @@ from file_paths import MANIFEST_WELLBORE_VALID_PATH, TRAVERSAL_WELLBORE_VALID_PA
     MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH, TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH, MANIFEST_EMPTY_PATH, \
     TRAVERSAL_MANIFEST_EMPTY_PATH, MANIFEST_GENERIC_SCHEMA_PATH
 from libs.exceptions import EmptyManifestError
-from libs.traverse_manifest import ManifestTraversal
+from libs.traverse_manifest import ManifestTraversal, ManifestEntity
 
 
 class TestManifestTraversal:
@@ -36,7 +36,7 @@ class TestManifestTraversal:
         with open(manifest_schema_file) as f:
             manifest_schema = json.load(f)
         traversal = ManifestTraversal(
-            conf_manifest_file["execution_context"],
+            conf_manifest_file["execution_context"].get("manifest", []),
             manifest_schema
         )
         return traversal
@@ -60,8 +60,10 @@ class TestManifestTraversal:
                                 manifest_schema_file: str, traversal_manifest_file: str):
         with open(traversal_manifest_file) as f:
             traversal_manifest = json.load(f)
+        traversal_manifest = [ManifestEntity(**e) for e in traversal_manifest]
         manifest_records = manifest_traversal.traverse_manifest()
-        assert manifest_records == traversal_manifest
+        for m in manifest_records:
+            assert m in traversal_manifest, f"Expected {traversal_manifest}\nGot    {manifest_records}"
 
     @pytest.mark.parametrize(
         "manifest_file,manifest_schema_file,traversal_manifest_file",
diff --git a/tests/plugin-unit-tests/test_operators_r3.py b/tests/plugin-unit-tests/test_operators_r3.py
index 0bada1d2923cc671acbd2d8d385dd126da77e88b..7553f24cdab64bc0de703b25ec6945e22383fba5 100644
--- a/tests/plugin-unit-tests/test_operators_r3.py
+++ b/tests/plugin-unit-tests/test_operators_r3.py
@@ -32,7 +32,7 @@ import requests
 
 from file_paths import (
     MANIFEST_WELLBORE_VALID_PATH,
-    SEARCH_VALID_RESPONSE_PATH, MANIFEST_GENERIC_SCHEMA_PATH)
+    SEARCH_VALID_RESPONSE_PATH, MANIFEST_GENERIC_SCHEMA_PATH, MANIFEST_BATCH_WELLBORE_VALID_PATH)
 from operators.process_manifest_r3 import ProcessManifestOperatorR3, SchemaValidator, \
     ManifestProcessor
 from operators.search_record_id import SearchRecordIdOperator
@@ -56,6 +56,17 @@ class MockStorageResponse(requests.Response):
 
 class TestOperators(object):
 
+    def _create_batch_task(self, operator: ClassVar[CustomOperator]) -> (CustomOperator, dict):
+        with open(MANIFEST_BATCH_WELLBORE_VALID_PATH) as f:
+            conf = json.load(f)
+        dag = DAG(dag_id='batch_osdu_ingest', start_date=datetime.now())
+        task: CustomOperator = operator(dag=dag, task_id='anytask')
+        ti = TaskInstance(task=task, execution_date=datetime.now())
+
+        context = ti.get_template_context()
+        context["dag_run"] = MockDagRun(conf)
+        return task, context
+
     def _create_task(self, operator: ClassVar[CustomOperator]) -> (CustomOperator, dict):
         with open(MANIFEST_WELLBORE_VALID_PATH) as f:
             conf = json.load(f)
@@ -73,7 +84,9 @@ class TestOperators(object):
             with open(MANIFEST_GENERIC_SCHEMA_PATH) as f:
                 manifest_schema = json.load(f)
             return manifest_schema
+
         monkeypatch.setattr(SchemaValidator, "get_schema", _get_common_schema)
+        monkeypatch.setattr(SchemaValidator, "_validate_against_schema", lambda *args, **kwargs: None)
         monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj, entities: entities)
         monkeypatch.setattr(ManifestProcessor, "save_record_to_storage",
                             lambda obj, headers, request_data: MockStorageResponse())
@@ -84,6 +97,25 @@ class TestOperators(object):
         task.pre_execute(context)
         task.execute(context)
 
+    def test_process_manifest_r3_operator_batch(self, monkeypatch):
+
+        def _get_common_schema(*args, **kwargs):
+            with open(MANIFEST_GENERIC_SCHEMA_PATH) as f:
+                manifest_schema = json.load(f)
+            return manifest_schema
+
+        monkeypatch.setattr(SchemaValidator, "get_schema", _get_common_schema)
+        monkeypatch.setattr(SchemaValidator, "_validate_against_schema", lambda *args, **kwargs: None)
+        monkeypatch.setattr(SchemaValidator, "validate_manifest", lambda obj, entities: entities)
+        monkeypatch.setattr(ManifestProcessor, "save_record_to_storage",
+                            lambda obj, headers, request_data: MockStorageResponse())
+        monkeypatch.setattr(FileHandler, "upload_file",
+                            lambda *args, **kwargs: "test")
+
+        task, context = self._create_batch_task(ProcessManifestOperatorR3)
+        task.pre_execute(context)
+        task.execute(context)
+
     def _test_update_status_operator(self, monkeypatch, status: UpdateStatusOperator.prev_ti_state):
         monkeypatch.setattr(UpdateStatusOperator, "get_previous_ti_statuses",
                             lambda obj, context: status)
diff --git a/tests/plugin-unit-tests/test_refresh_token.py b/tests/plugin-unit-tests/test_refresh_token.py
index 49d98a2c5569d962e870a652ef23f46f24743595..f468c3c618465d5bfa846b8e65ee3fc7d526cb7e 100644
--- a/tests/plugin-unit-tests/test_refresh_token.py
+++ b/tests/plugin-unit-tests/test_refresh_token.py
@@ -23,7 +23,7 @@ from google.oauth2 import service_account
 
 sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
 
-from libs.refresh_token import AirflowTokenRefresher, TokenRefresher
+from libs.refresh_token import AirflowTokenRefresher
 from mock_providers import get_test_credentials
 
 
@@ -42,12 +42,12 @@ class TestAirflowTokenRefresher:
             "test"
         ]
     )
-    def test_access_token_cached(self, token_refresher: TokenRefresher, access_token: str):
+    def test_access_token_cached(self, token_refresher: AirflowTokenRefresher, access_token: str):
         """
         Check if access token stored in Airflow Variables after refreshing it.
         """
         token_refresher.refresh_token()
-        assert token_refresher.airflow_variables.get("access_token") == access_token
+        assert token_refresher.airflow_variables.get("core__auth__access_token") == access_token
 
     @pytest.mark.parametrize(
         "access_token",
@@ -56,7 +56,7 @@ class TestAirflowTokenRefresher:
             "aaaa"
         ]
     )
-    def test_authorization_header(self, token_refresher: TokenRefresher, access_token: str):
+    def test_authorization_header(self, token_refresher: AirflowTokenRefresher, access_token: str):
         """
         Check if Authorization header is 'Bearer <access_token>'
         """
@@ -73,7 +73,7 @@ class TestAirflowTokenRefresher:
     def test_refresh_token_no_cached_variable(
         self,
         monkeypatch,
-        token_refresher: TokenRefresher,
+        token_refresher: AirflowTokenRefresher,
         access_token: str,
     ):
         """
diff --git a/tests/plugin-unit-tests/test_schema_validator_r3.py b/tests/plugin-unit-tests/test_schema_validator_r3.py
index e35e66c456e36d0db0fb8c5705e5ca7dcb278c81..a3201e18f9c869c2d4fc182edc09dfb858f8d685 100644
--- a/tests/plugin-unit-tests/test_schema_validator_r3.py
+++ b/tests/plugin-unit-tests/test_schema_validator_r3.py
@@ -36,7 +36,7 @@ from file_paths import (
     MANIFEST_SEISMIC_TRACE_DATA_VALID_PATH,
     MANIFEST_WELLBORE_VALID_PATH,
     MANIFEST_GENERIC_PATH,
-    MANIFEST_GENERIC_SCHEMA_PATH,
+    MANIFEST_NEW_GENERIC_SCHEMA_PATH,
     SCHEMA_WELLBORE_VALID_PATH,
     TRAVERSAL_WELLBORE_VALID_PATH,
     TRAVERSAL_SEISMIC_TRACE_DATA_VALID_PATH,
@@ -44,11 +44,12 @@ from file_paths import (
 )
 from mock_responses import MockSchemaResponse
 from libs.context import Context
+from libs.traverse_manifest import ManifestEntity
 from libs.refresh_token import AirflowTokenRefresher
 from libs.exceptions import EmptyManifestError, NotOSDUSchemaFormatError
 import pytest
 
-from libs.validate_schema import SchemaValidator
+from libs.validation.validate_schema import SchemaValidator
 
 TENANT = "opendes"
 
@@ -111,28 +112,11 @@ class TestSchemaValidator:
         monkeypatch.setattr(schema_validator, "get_schema", self.mock_get_schema)
         with open(traversal_manifest_file_path) as f:
             manifest_file = json.load(f)
+
+        manifest_file = [ManifestEntity(entity=e["entity"], schema=e["schema"]) for e in manifest_file]
         validated_records = schema_validator.validate_manifest(manifest_file)
         assert len(manifest_file) == len(validated_records)
 
-    @pytest.mark.parametrize(
-        "manifest_file,traversal_manifest_file,schema_file",
-        [
-            pytest.param(
-                MANIFEST_EMPTY_PATH, TRAVERSAL_MANIFEST_EMPTY_PATH, None,
-                id="Empty Manifest"),
-        ]
-    )
-    def test_schema_validator_empty_manifest(self,
-                                             monkeypatch,
-                                             schema_validator: SchemaValidator,
-                                             manifest_file: str,
-                                             traversal_manifest_file: str,
-                                             schema_file: str):
-        with open(traversal_manifest_file) as f:
-            manifest_file = json.load(f)
-        with pytest.raises(EmptyManifestError):
-            schema_validator.validate_manifest(manifest_file)
-
     @pytest.mark.parametrize(
         "traversal_manifest_file,schema_file",
         [
@@ -149,6 +133,7 @@ class TestSchemaValidator:
                                               schema_file: str):
         with open(traversal_manifest_file) as f:
             manifest_file = json.load(f)
+        manifest_file = [ManifestEntity(entity=e["entity"], schema=e["schema"]) for e in manifest_file]
         with pytest.raises(NotOSDUSchemaFormatError):
             schema_validator.validate_manifest(manifest_file)
 
@@ -201,7 +186,7 @@ class TestSchemaValidator:
         [
             pytest.param(
                 MANIFEST_GENERIC_PATH,
-                MANIFEST_GENERIC_SCHEMA_PATH,
+                MANIFEST_NEW_GENERIC_SCHEMA_PATH,
                 id="Valid generic manifest"),
         ]
     )
@@ -220,4 +205,16 @@ class TestSchemaValidator:
         with open(manifest_file) as f:
             manifest_content = json.load(f)
         monkeypatch.setattr(schema_validator, "get_schema", mock_get_schema)
-        schema_validator.validate_common_schema({"manifest": manifest_content})
+        schema_validator.validate_common_schema(manifest_content)
+
+    def test_delete_refs(self):
+        context = Context(app_key="", data_partition_id="")
+        validator = SchemaValidator(
+            "",
+            AirflowTokenRefresher(get_test_credentials()),
+            context
+        )
+        manifest = 	{'manifest': {'ReferenceData': [], 'MasterData': [], 'kind': 'osdu:wks:Manifest:1.0.0', 'Data': {}}}
+        schema = { "x-osdu-license": "Copyright 2021, The Open Group \\nLicensed under the Apache License, Version 2.0 (the \"License\"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 . Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.", "$id": "https://schema.osdu.opengroup.org/json/manifest/Manifest.1.0.0.json", "$schema": "http://json-schema.org/draft-07/schema#", "x-osdu-schema-source": "osdu:wks:Manifest:1.0.0", "title": "Load Manifest Schema", "description": "Load manifest applicable for all types defined as 'kind', i.e. registered as schemas with the Schema Service. It supports loading of individual 'records' of any group-type or combinations. The load sequence follows a well-defined sequence. The 'ReferenceData' array is processed first (if populated). The 'MasterData' array is processed second (if populated) second. The 'Data' structure is processed last (if populated). Inside the 'Data' property the 'Datasets' array is processed first, followed by the 'WorkProductComponents' array, the 'WorkProduct' is processed last. Any arrays are ordered. should there be interdependencies, the dependent items must be placed behind their relationship targets, e.g. a master-data Well record must placed in the 'MasterData' array before its Wellbores.", "type": "object", "properties": { "kind": { "description": "The schema identification for the manifest record following the pattern {Namespace}:{Source}:{Type}:{VersionMajor}.{VersionMinor}.{VersionPatch}. The versioning scheme follows the semantic versioning, https://semver.org/.", "title": "Manifest  Kind", "type": "string", "pattern": "^[\\w\\-\\.]+:[\\w\\-\\.]+:[\\w\\-\\.]+:[0-9]+.[0-9]+.[0-9]+$", "example": "osdu:wks:Manifest:1.0.0" }, "ReferenceData": { "description": "Reference-data are submitted as an array of records.", "type": "array", "items": { "$ref": "GenericReferenceData.1.0.0.json" } }, "MasterData": { "description": "Master-data are submitted as an array of records.", "type": "array", "items": { "$ref": "GenericMasterData.1.0.0.json" } }, "Data": { "description": "Manifest schema for work-product, work-product-component, dataset ensembles. The items in 'Datasets' are processed first since they are referenced by 'WorkProductComponents' ('data.DatasetIDs[]' and 'data.Artefacts[].ResourceID'). The WorkProduct is processed last collecting the WorkProductComponents.", "type": "object", "properties": { "WorkProduct": { "description": "The work-product component capturing the work-product-component records belonging to this loading/ingestion transaction.", "$ref": "GenericWorkProduct.1.0.0.json" }, "WorkProductComponents": { "description": "The list of work-product-components records. The record ids are internal surrogate keys enabling the association of work-product-component records with the work-product records.", "type": "array", "items": { "$ref": "GenericWorkProductComponent.1.0.0.json" } }, "Datasets": { "description": "The list of 'Datasets' or data containers holding the actual data. The record ids are usually internal surrogate keys enabling the association of dataset records with work-product-component records, namely via 'DatasetIDs' and 'Artefacts.ResourceID' (both referring to 'dataset' group-type entity types).", "type": "array", "items": { "$ref": "GenericDataset.1.0.0.json" } } } } }, "x-osdu-inheriting-from-kind": [] }
+        validator._clear_data_fields(schema)
+        validator._validate_against_schema(schema, manifest)
diff --git a/tests/plugin-unit-tests/test_search_client.py b/tests/plugin-unit-tests/test_search_client.py
new file mode 100644
index 0000000000000000000000000000000000000000..24aa14c9db17d9a09c079183f02f3a46ece763e0
--- /dev/null
+++ b/tests/plugin-unit-tests/test_search_client.py
@@ -0,0 +1,74 @@
+#  Copyright 2021 Google LLC
+#  Copyright 2021 EPAM Systems
+#
+#  Licensed under the Apache License, Version 2.0 (the "License");
+#  you may not use this file except in compliance with the License.
+#  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+import http
+import io
+import json
+import os
+import sys
+
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/plugins")
+sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
+
+import pytest
+import responses
+
+from libs.context import Context
+from libs.refresh_token import AirflowTokenRefresher
+from libs.search_client import SearchClient, SearchResponse
+from mock_providers import get_test_credentials
+
+
+class TestSearchClient:
+
+    BASE_TEST_URL = "http://search_service_url/query"
+    PARTITION_ID = "test_partition_id"
+
+    @pytest.fixture
+    def search_client(self):
+        context = Context(data_partition_id=self.PARTITION_ID, app_key="")
+        search_client = SearchClient(self.BASE_TEST_URL,
+                                     AirflowTokenRefresher(get_test_credentials()), context)
+        return search_client
+
+    @pytest.mark.parametrize("kind, query_str, limit, results, total_count", [
+        pytest.param("*:*:*:*", "some_query", 20, [], 10),
+        pytest.param("opendes:osdu:File:1.0.0", "id:(test1 OR test2)", 10, [{}, {}], 5)
+    ])
+    @responses.activate
+    def test_query_record_with_defaults(self, search_client, kind: str, query_str: str, limit: int,
+                                        results: list, total_count: int):
+
+        responses.add(responses.POST,
+                      f"{self.BASE_TEST_URL}",
+                      json={
+                          "results": results,
+                          "aggregations": None,
+                          "totalCount": total_count
+                      },
+                      status=http.HTTPStatus.OK)
+
+        search_response = search_client.query_records(kind, query_str, limit)
+
+        expected_body = {
+            "kind": kind,
+            "query": query_str,
+            "limit": limit,
+            "offset": 0,
+        }
+
+        assert expected_body == json.loads(responses.calls[0].request.body)
+        assert results == search_response.results
+        assert total_count == search_response.total_count
diff --git a/tests/plugin-unit-tests/test_search_record_id.py b/tests/plugin-unit-tests/test_search_record_id.py
index b8003982447da9fe96ba1a7797b94d403f0e704c..d13938e9f5314fdf67b65d440dccfa5f08a43681 100644
--- a/tests/plugin-unit-tests/test_search_record_id.py
+++ b/tests/plugin-unit-tests/test_search_record_id.py
@@ -12,8 +12,7 @@
 #  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
-
-
+import json
 import os
 import sys
 import http
@@ -26,13 +25,13 @@ sys.path.append(f"{os.getenv('AIRFLOW_SRC_DIR')}/dags")
 from mock_providers import get_test_credentials
 from file_paths import (
     SEARCH_INVALID_RESPONSE_PATH,
-    SEARCH_VALID_RESPONSE_PATH
-)
+    SEARCH_VALID_RESPONSE_PATH,
+    SEARCH_EXTRACTED_IDS_PATH)
 from libs.exceptions import RecordsNotSearchableError
 from libs.context import Context
 from libs.refresh_token import AirflowTokenRefresher
 from tenacity import stop_after_attempt
-from libs.search_record_ids import SearchId
+from libs.search_record_ids import SearchId, ExtendedSearchId
 from mock_responses import MockSearchResponse
 
 
@@ -133,3 +132,27 @@ class TestManifestProcessor:
         with pytest.raises(ValueError):
             SearchId("http://test", record_ids, AirflowTokenRefresher(get_test_credentials()),
                      Context(app_key="", data_partition_id=""))
+
+    @pytest.mark.parametrize(
+        "record_ids,search_response_path,extracted_ids_path",
+        [
+            pytest.param(
+                ["test"],
+                SEARCH_VALID_RESPONSE_PATH,
+                SEARCH_EXTRACTED_IDS_PATH
+            )
+        ]
+    )
+    def test_search_found_all_records(self, monkeypatch, record_ids: list,
+                                      search_response_path: str,
+                                      extracted_ids_path: str):
+        self.mock_storage_response(monkeypatch, search_response_path, total_count=len(record_ids))
+        with open(search_response_path) as f:
+            response = json.load(f)
+        id_searcher = ExtendedSearchId("http://test", record_ids,
+                                       AirflowTokenRefresher(get_test_credentials()),
+                                       Context(app_key="", data_partition_id=""))
+        record_ids = id_searcher._extract_id_from_response(response)
+        with open(extracted_ids_path) as f:
+            extracted_ids = json.load(f)
+        assert set(record_ids) == set(extracted_ids)
diff --git a/tests/providers-unit-tests/test_gcp_blob_storage_client.py b/tests/providers-unit-tests/test_gcp_blob_storage_client.py
index f80e94bfe7c117096be9e4a6326f0d4283e177e6..856180c6c3fefb593e3716a9a2fef1c2e81ef83b 100644
--- a/tests/providers-unit-tests/test_gcp_blob_storage_client.py
+++ b/tests/providers-unit-tests/test_gcp_blob_storage_client.py
@@ -144,7 +144,7 @@ class TestGoogleCloudStorageClient:
 
         client_mock.bucket.assert_called_with(bucket_name)
         bucket_mock.get_blob.assert_called_with(blob_name)
-        blob_mock.download_as_bytes.assert_called_with()
+        blob_mock.download_as_string.assert_called_with()
 
     @pytest.mark.parametrize("uri, bucket_name, blob_name, content_type", [
         pytest.param("gs://bucket_test/name_test", "bucket_test", "name_test", "text/html"),
diff --git a/tests/set_airflow_env.sh b/tests/set_airflow_env.sh
index 90c48a531fe548c44b348bd976cc21d4d4c02a86..e8751340d8e888a708f79d230496e668026056b5 100755
--- a/tests/set_airflow_env.sh
+++ b/tests/set_airflow_env.sh
@@ -17,14 +17,13 @@ pip install --upgrade google-api-python-client
 pip install dataclasses
 pip install jsonschema
 pip install google
+pip install toposort
 pip install google-cloud-storage
 pip install deepdiff
 pip install azure-identity
 pip install azure-keyvault-secrets
 pip install msal
 pip install python-keycloak
-export ACL='{"viewers": ["foo"],"owners": ["foo"]}'
-export LEGAL='{"legaltags": ["foo"], "otherRelevantDataCountries": ["FR", "US", "CA"],"status": "compliant"}'
 export WORKFLOW_URL="http://127.0.0.1:5000"
 export UPDATE_STATUS_URL="http://127.0.0.1:5000/wf/us"
 export STORAGE_URL="http://127.0.0.1:5000/st"
@@ -42,35 +41,27 @@ sed -i 's/load_examples = True/load_examples = False/'  /usr/local/airflow/airfl
 # turn on all dags
 sed -i 's/dags_are_paused_at_creation = True/dags_are_paused_at_creation = False/'  /usr/local/airflow/airflow.cfg
 
-airflow variables -s storage_url $STORAGE_URL
-airflow variables -s id_token 	http://127.0.0.1:5000/storage
-airflow variables -s acl "$ACL"
-airflow variables -s data_partition_id 	odes
-airflow variables -s entitlements_module_name entitlements_client
-airflow variables -s legal "$LEGAL"
-airflow variables -s provider gcp
-airflow variables -s record_kind "odes:osdu:file:0.2.0"
-airflow variables -s schema_version "0.2.0"
-airflow variables -s workflow_url $WORKFLOW_URL
-airflow variables -s file_service_url $LOCALHOST
-airflow variables -s update_status_url $UPDATE_STATUS_URL
-airflow variables -s search_url $SEARCH_URL
-airflow variables -s schema_service_url  $LOCALHOST
-airflow variables -s dataload_config_path $DATALOAD_CONFIG_PATH
-airflow variables -s search_query_ep sr/qr
-airflow variables -s access_token test
-airflow variables -s "sa-file-osdu" "test"
+airflow variables -s core__service__storage__url $STORAGE_URL
+airflow variables -s core__provider gcp
+airflow variables -s core__service__workflow__host $WORKFLOW_URL
+airflow variables -s core__service__file__host $LOCALHOST
+airflow variables -s core__service__workflow__url $UPDATE_STATUS_URL
+airflow variables -s core__service__search__url $SEARCH_URL
+airflow variables -s core__service__schema__url  $LOCALHOST
+airflow variables -s core__config__dataload_config_path $DATALOAD_CONFIG_PATH
+airflow variables -s core__auth__access_token test
+airflow variables -s core__ingestion__batch_count 3
 
 airflow connections -a --conn_id workflow --conn_uri $WORKFLOW_CONN_ID
-airflow connections -a --conn_id google_cloud_storage --conn_uri $WORKFLOW_CONN_ID
+airflow connections -a --conn_id google_cloud_storage --conn_uri $LOCALHOST
 
 mkdir -p /usr/local/airflow/dags/
 cp -rf src/* /usr/local/airflow/
 cp -r tests/end-to-end-tests/mock-external-apis /mock-server
 cp -r tests/end-to-end-tests/mock-data /mock-server/mock-data
 
-cp tests/end-to-end-tests/{test-osdu-ingest-r2-success.sh,test-osdu-ingest-r2-fail.sh,test-default-ingest-{success,fail}.sh} /mock-server/
+cp tests/end-to-end-tests/{test-osdu-ingest-r2-success.sh,test-osdu-ingest-r2-fail.sh} /mock-server/
 cp tests/end-to-end-tests/osdu_api_config.yaml /mock-server/
 cp tests/*.py /mock-server/
 
-chmod +x /mock-server/{test-osdu-ingest-r2-success.sh,test-osdu-ingest-r2-fail.sh,test-default-ingest-{success,fail}.sh}
+chmod +x /mock-server/{test-osdu-ingest-r2-success.sh,test-osdu-ingest-r2-fail.sh}
diff --git a/tests/test_dags.py b/tests/test_dags.py
index 3fe61f3d20b0a156a47a8d7dbcd5eb97d9fbae62..2c9ebe811f3e7fdaef4dfb10d69373ce9c83256d 100644
--- a/tests/test_dags.py
+++ b/tests/test_dags.py
@@ -27,8 +27,6 @@ class DagStatus(enum.Enum):
 
 OSDU_INGEST_SUCCESS_SH = "/mock-server/./test-osdu-ingest-r2-success.sh"
 OSDU_INGEST_FAIL_SH = "/mock-server/./test-osdu-ingest-r2-fail.sh"
-DEFAULT_INGEST_SUCCESS_SH = "/mock-server/./test-default-ingest-success.sh"
-DEFAULT_INGEST_FAIL_SH = "/mock-server/./test-default-ingest-fail.sh"
 
 with open("/tmp/osdu_ingest_result", "w") as f:
     f.close()
@@ -79,5 +77,3 @@ def test_dag_fail(dag_name, script):
 
 test_dag_success("Osdu_ingest_r2", OSDU_INGEST_SUCCESS_SH)
 test_dag_fail("Osdu_ingest_r2", OSDU_INGEST_FAIL_SH)
-test_dag_success("Default_ingest", DEFAULT_INGEST_SUCCESS_SH)
-test_dag_fail("Default_ingest", DEFAULT_INGEST_FAIL_SH)