You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sdap.apache.org by tl...@apache.org on 2021/04/09 01:17:19 UTC

[incubator-sdap-ingester] branch ascending_latitudes created (now e101a17)

This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a change to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git.


      at e101a17  temporarilly remove too big test granule

This branch includes the following new commits:

     new 36b87d0  SDAP-234: Add more test coverage. (#1)
     new 2c991c5  SDAP-254, SDAP-255, SDAP-256: Fix bug where ingestion history is not saved, fix bug where messages published to RabbitMQ are incorrect, fix bug where bad collection config file crashes app (#3)
     new 2619a7b  SDAP-245: Move granule ingester code into this repo (#2)
     new cd42785  SDAP-237 Dockerize Collection Manager (#4)
     new 242de66  SDAP-259: The Collection Manager now reloads the Collections Config file on an interval instead of watching for filesystem events (#5)
     new 5256cdd  SDAP-247: config-operator unit tests and support for git username/token  (#6)
     new d748429  SDAP-269: Switch to using aio-pika in collection-manager to maintain an asynchronous connection to RabbitMQ (#7)
     new bf8c911  SDAP-270: Fix bad solr endpoints in solr history manager (#8)
     new 4f7fdf2  Fix syntax error (#9)
     new cf30aad  add a on.update kopf event for crd updates
     new a9a0629  SDAP-271 Cassandra authentication support (#11)
     new ab6998b  SDAP-273: Configure max threads in Granule Ingester (#13)
     new 7590df9  SDAP-266: add README note on synchronization of configmap with local path (#14)
     new f9462ed  SDAP-272: Support connecting to Solr through Zk in Granule Ingester (#12)
     new caa296a  SDAP-277: Improved error handling in Granule Ingester (#15)
     new 3fde409  SDAP-279: Collection Manager should poll filesystem for file updates (#17)
     new 0aa6121  SDAP-280: Collection Manager to talk to Ingestion History asynchronously (#16)
     new e42ef8b  SDAP-282, SDAP-284: Support configuring dimension names and projection during ingestion, support hierarchical directory structures (#19)
     new 97d5a2c  SDAP-291: Fix netcdf parsing error by freezing pandas at 1.0.4 (#20)
     new 0586175  SDAP-291: Fix netcdf parsing error by freezing pandas at 1.0.4 (#21)
     new dba7a01  SDAP-286: Add processor module to Granule Ingester to properly handle longitudes between 180-360deg (#23)
     new 31bff5e  SDAP-288: S3 ingestion support (#24)
     new 0d87acb  SDAP-293: Automatic Kelvin-to-Celsius conversion (#22)
     new aa82958  SDAP-295: Include dataset name hash in tile ID generation (#25)
     new 528ba21  SDAP-297: Update Collections Config docs to match latest schema (#26)
     new 7448280  SDAP-300: Fix bug that prevented collection manager from seeing files in a directory when the path had no glob-style wildcard character. Also add back support for scanning dirs recursively. (#27)
     new 677fb57  KelvinToCelsius converter can now parse 'units' or 'Units' field and exits gracefully if none exists (#28)
     new d705545  SDAP-302: Fix bug where the Collection Manager would crash if the Collections Config was updated while Collection Manager was actively scanning S3 directories (#29)
     new e13d7cc  add processor to make latitude ascending
     new e2390ce  add lfs support for test unit file ()nc, nc4, h5
     new aea467d  replace hls test data with smaller one
     new 54b8eec  restore files from lfs
     new e101a17  temporarilly remove too big test granule

The 33 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


[incubator-sdap-ingester] 03/33: SDAP-245: Move granule ingester code into this repo (#2)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 2619a7bce43c31f34465c49e38a55f76b215c040
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Mon Jun 22 17:49:38 2020 -0700

    SDAP-245: Move granule ingester code into this repo (#2)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 .gitattributes                                     |   1 -
 granule_ingester/.gitignore                        |   9 +
 granule_ingester/README.md                         |  34 +++
 granule_ingester/conda-requirements.txt            |  10 +
 granule_ingester/docker/Dockerfile                 |  21 ++
 granule_ingester/docker/entrypoint.sh              |  10 +
 granule_ingester/docker/install_nexusproto.sh      |  20 ++
 granule_ingester/granule_ingester/__init__.py      |   0
 .../granule_ingester/consumer/Consumer.py          |  88 +++++++
 .../granule_ingester/consumer/__init__.py          |   1 +
 .../granule_loaders/GranuleLoader.py               |  71 ++++++
 .../granule_ingester/granule_loaders/__init__.py   |   1 +
 .../granule_ingester/healthcheck/HealthCheck.py    |  22 ++
 .../granule_ingester/healthcheck/__init__.py       |   1 +
 granule_ingester/granule_ingester/main.py          | 118 +++++++++
 .../granule_ingester/pipeline/Modules.py           |  15 ++
 .../granule_ingester/pipeline/Pipeline.py          | 158 ++++++++++++
 .../granule_ingester/pipeline/__init__.py          |   2 +
 .../granule_ingester/processors/EmptyTileFilter.py |  42 ++++
 .../granule_ingester/processors/GenerateTileId.py  |  32 +++
 .../granule_ingester/processors/TileProcessor.py   |  23 ++
 .../processors/TileSummarizingProcessor.py         |  98 ++++++++
 .../granule_ingester/processors/__init__.py        |   5 +
 .../granule_ingester/processors/kelvintocelsius.py |  31 +++
 .../reading_processors/EccoReadingProcessor.py     |  64 +++++
 .../reading_processors/GridReadingProcessor.py     |  53 +++++
 .../reading_processors/SwathReadingProcessor.py    |  47 ++++
 .../reading_processors/TileReadingProcessor.py     |  81 +++++++
 .../TimeSeriesReadingProcessor.py                  |  83 +++++++
 .../processors/reading_processors/__init__.py      |   5 +
 .../slicers/SliceFileByDimension.py                |  55 +++++
 .../slicers/SliceFileByStepSize.py                 |  55 +++++
 .../slicers/SliceFileByTilesDesired.py             |  68 ++++++
 .../granule_ingester/slicers/TileSlicer.py         |  56 +++++
 .../granule_ingester/slicers/__init__.py           |   2 +
 .../granule_ingester/writers/CassandraStore.py     |  78 ++++++
 .../granule_ingester/writers/DataStore.py          |  13 +
 .../granule_ingester/writers/MetadataStore.py      |  11 +
 .../granule_ingester/writers/SolrStore.py          | 152 ++++++++++++
 .../granule_ingester/writers/__init__.py           |   4 +
 granule_ingester/requirements.txt                  |   3 +
 granule_ingester/setup.py                          |  34 +++
 granule_ingester/tests/__init__.py                 |   0
 .../tests/config_files/analysed_sst.yml            |  16 ++
 .../config_files/ingestion_config_testfile.yaml    |  17 ++
 ...4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc | Bin 0 -> 1057327 bytes
 granule_ingester/tests/granules/OBP_2017_01.nc     | Bin 0 -> 2110135 bytes
 granule_ingester/tests/granules/OBP_native_grid.nc | Bin 0 -> 1285094 bytes
 .../SMAP_L2B_SSS_04892_20160101T005507_R13080.h5   | Bin 0 -> 18672352 bytes
 granule_ingester/tests/granules/THETA_199201.nc    | Bin 0 -> 4255957 bytes
 granule_ingester/tests/granules/empty_mur.nc4      | Bin 0 -> 60937 bytes
 .../tests/granules/not_empty_ascatb.nc4            | Bin 0 -> 78036 bytes
 .../tests/granules/not_empty_avhrr.nc4             | Bin 0 -> 49511 bytes
 granule_ingester/tests/granules/not_empty_ccmp.nc  | Bin 0 -> 206870 bytes
 granule_ingester/tests/granules/not_empty_mur.nc4  | Bin 0 -> 60907 bytes
 granule_ingester/tests/granules/not_empty_smap.h5  | Bin 0 -> 3000192 bytes
 granule_ingester/tests/granules/not_empty_wswm.nc  | Bin 0 -> 1041568 bytes
 granule_ingester/tests/pipeline/__init__.py        |   0
 granule_ingester/tests/pipeline/test_Pipeline.py   | 104 ++++++++
 granule_ingester/tests/processors/__init__.py      |   0
 .../tests/processors/test_GenerateTileId.py        |  22 ++
 .../tests/reading_processors/__init__.py           |   0
 .../test_EccoReadingProcessor.py                   |  64 +++++
 .../test_GridReadingProcessor.py                   | 265 +++++++++++++++++++++
 .../test_SwathReadingProcessor.py                  |  74 ++++++
 .../test_TileReadingProcessor.py                   |  29 +++
 .../test_TimeSeriesReadingProcessor.py             |  86 +++++++
 granule_ingester/tests/slicers/__init__.py         |   0
 .../tests/slicers/test_SliceFileByDimension.py     | 122 ++++++++++
 .../tests/slicers/test_SliceFileByStepSize.py      | 105 ++++++++
 .../tests/slicers/test_SliceFileByTilesDesired.py  |  88 +++++++
 granule_ingester/tests/slicers/test_TileSlicer.py  |  68 ++++++
 granule_ingester/tests/writers/__init__.py         |   0
 granule_ingester/tests/writers/test_SolrStore.py   |  54 +++++
 74 files changed, 2790 insertions(+), 1 deletion(-)

diff --git a/.gitattributes b/.gitattributes
deleted file mode 100644
index 9abd205..0000000
--- a/.gitattributes
+++ /dev/null
@@ -1 +0,0 @@
-*.nc filter=lfs diff=lfs merge=lfs -text
diff --git a/granule_ingester/.gitignore b/granule_ingester/.gitignore
new file mode 100644
index 0000000..5408b74
--- /dev/null
+++ b/granule_ingester/.gitignore
@@ -0,0 +1,9 @@
+.vscode
+.idea
+*.egg-info
+*__pycache__
+*.pytest_cache
+*.code-workspace
+.DS_STORE
+build
+dist
\ No newline at end of file
diff --git a/granule_ingester/README.md b/granule_ingester/README.md
new file mode 100644
index 0000000..112f52d
--- /dev/null
+++ b/granule_ingester/README.md
@@ -0,0 +1,34 @@
+# SDAP Granule Ingester
+
+The SDAP Granule Ingester is a service that reads from a RabbitMQ queue for
+YAML-formated string messages produced by the Collection Manager (`/collection_manager` 
+in this repo). For each message consumed, this service will read a granule file from
+disk and ingest it into SDAP by processing the granule and writing the resulting
+data to Cassandra and Solr.
+
+
+## Prerequisites
+
+Python 3.7
+
+## Building the service
+From `incubator-sdap-ingester/granule_ingester`, run:
+
+    $ python setup.py install
+    
+
+## Launching the service
+From `incubator-sdap-ingester/granule_ingester`, run:
+
+    $ python granule_ingester/main.py -h
+    
+## Running the tests
+From `incubator-sdap-ingester/granule_ingester`, run:
+
+    $ pip install pytest
+    $ pytest
+    
+## Building the Docker image
+From `incubator-sdap-ingester/granule_ingester`, run:
+
+    $ docker build . -f docker/Dockerfile -t nexusjpl/granule-ingester
diff --git a/granule_ingester/conda-requirements.txt b/granule_ingester/conda-requirements.txt
new file mode 100644
index 0000000..b2af149
--- /dev/null
+++ b/granule_ingester/conda-requirements.txt
@@ -0,0 +1,10 @@
+numpy==1.15.4
+scipy
+netcdf4==1.5.3
+pytz==2019.3
+xarray
+pyyaml==5.3.1
+requests==2.23.0
+aiohttp==3.6.2
+aio-pika
+tenacity
diff --git a/granule_ingester/docker/Dockerfile b/granule_ingester/docker/Dockerfile
new file mode 100644
index 0000000..4b25318
--- /dev/null
+++ b/granule_ingester/docker/Dockerfile
@@ -0,0 +1,21 @@
+FROM continuumio/miniconda3:4.8.2-alpine
+
+USER root
+
+ENV PATH="/opt/conda/bin:$PATH"
+
+RUN apk update --no-cache && apk add --no-cache --virtual .build-deps git openjdk8
+
+COPY /granule_ingester /sdap/granule_ingester
+COPY /setup.py /sdap/setup.py
+COPY /requirements.txt /sdap/requirements.txt
+COPY /conda-requirements.txt /sdap/conda-requirements.txt
+COPY /docker/install_nexusproto.sh /install_nexusproto.sh
+COPY /docker/entrypoint.sh /entrypoint.sh
+
+RUN ./install_nexusproto.sh
+RUN cd /sdap && python setup.py install
+
+RUN apk del .build-deps
+
+ENTRYPOINT ["/bin/sh", "/entrypoint.sh"]
\ No newline at end of file
diff --git a/granule_ingester/docker/entrypoint.sh b/granule_ingester/docker/entrypoint.sh
new file mode 100644
index 0000000..e6f7262
--- /dev/null
+++ b/granule_ingester/docker/entrypoint.sh
@@ -0,0 +1,10 @@
+#!/bin/sh
+
+python /sdap/granule_ingester/main.py \
+  $([[ ! -z "$RABBITMQ_HOST" ]] && echo --rabbitmq_host=$RABBITMQ_HOST) \
+  $([[ ! -z "$RABBITMQ_USERNAME" ]] && echo --rabbitmq_username=$RABBITMQ_USERNAME) \
+  $([[ ! -z "$RABBITMQ_PASSWORD" ]] && echo --rabbitmq_password=$RABBITMQ_PASSWORD) \
+  $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq_queue=$RABBITMQ_QUEUE) \
+  $([[ ! -z "$CASSANDRA_CONTACT_POINTS" ]] && echo --cassandra_contact_points=$CASSANDRA_CONTACT_POINTS) \
+  $([[ ! -z "$CASSANDRA_PORT" ]] && echo --cassandra_port=$CASSANDRA_PORT) \
+  $([[ ! -z "$SOLR_HOST_AND_PORT" ]] && echo --solr_host_and_port=$SOLR_HOST_AND_PORT)
diff --git a/granule_ingester/docker/install_nexusproto.sh b/granule_ingester/docker/install_nexusproto.sh
new file mode 100755
index 0000000..7ba2cee
--- /dev/null
+++ b/granule_ingester/docker/install_nexusproto.sh
@@ -0,0 +1,20 @@
+set -e
+
+APACHE_NEXUSPROTO="https://github.com/apache/incubator-sdap-nexusproto.git"
+MASTER="master"
+
+GIT_REPO=${1:-$APACHE_NEXUSPROTO}
+GIT_BRANCH=${2:-$MASTER}
+
+mkdir nexusproto
+cd nexusproto
+git init
+git pull ${GIT_REPO} ${GIT_BRANCH}
+
+./gradlew pythonInstall --info
+
+./gradlew install --info
+
+rm -rf /root/.gradle
+cd ..
+rm -rf nexusproto
diff --git a/granule_ingester/granule_ingester/__init__.py b/granule_ingester/granule_ingester/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/granule_ingester/consumer/Consumer.py b/granule_ingester/granule_ingester/consumer/Consumer.py
new file mode 100644
index 0000000..75d347a
--- /dev/null
+++ b/granule_ingester/granule_ingester/consumer/Consumer.py
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 logging
+
+import aio_pika
+
+from granule_ingester.healthcheck import HealthCheck
+from granule_ingester.pipeline import Pipeline
+
+logger = logging.getLogger(__name__)
+
+
+class Consumer(HealthCheck):
+
+    def __init__(self,
+                 rabbitmq_host,
+                 rabbitmq_username,
+                 rabbitmq_password,
+                 rabbitmq_queue,
+                 data_store_factory,
+                 metadata_store_factory):
+        self._rabbitmq_queue = rabbitmq_queue
+        self._data_store_factory = data_store_factory
+        self._metadata_store_factory = metadata_store_factory
+
+        self._connection_string = "amqp://{username}:{password}@{host}/".format(username=rabbitmq_username,
+                                                                                password=rabbitmq_password,
+                                                                                host=rabbitmq_host)
+        self._connection = None
+
+    async def health_check(self) -> bool:
+        try:
+            connection = await self._get_connection()
+            await connection.close()
+            return True
+        except:
+            logger.error("Cannot connect to RabbitMQ! Connection string was {}".format(self._connection_string))
+            return False
+
+    async def _get_connection(self):
+        return await aio_pika.connect_robust(self._connection_string)
+
+    async def __aenter__(self):
+        self._connection = await self._get_connection()
+        return self
+
+    async def __aexit__(self, exc_type, exc_val, exc_tb):
+        if self._connection:
+            await self._connection.close()
+
+    @staticmethod
+    async def _received_message(message: aio_pika.IncomingMessage,
+                                data_store_factory,
+                                metadata_store_factory):
+        logger.info("Received a job from the queue. Starting pipeline.")
+        try:
+            config_str = message.body.decode("utf-8")
+            logger.debug(config_str)
+            pipeline = Pipeline.from_string(config_str=config_str,
+                                            data_store_factory=data_store_factory,
+                                            metadata_store_factory=metadata_store_factory)
+            await pipeline.run()
+            message.ack()
+        except Exception as e:
+            message.reject(requeue=True)
+            logger.error("Processing message failed. Message will be re-queued. The exception was:\n{}".format(e))
+
+    async def start_consuming(self):
+        channel = await self._connection.channel()
+        await channel.set_qos(prefetch_count=1)
+        queue = await channel.declare_queue(self._rabbitmq_queue, durable=True)
+
+        async with queue.iterator() as queue_iter:
+            async for message in queue_iter:
+                await self._received_message(message, self._data_store_factory, self._metadata_store_factory)
diff --git a/granule_ingester/granule_ingester/consumer/__init__.py b/granule_ingester/granule_ingester/consumer/__init__.py
new file mode 100644
index 0000000..35d075b
--- /dev/null
+++ b/granule_ingester/granule_ingester/consumer/__init__.py
@@ -0,0 +1 @@
+from granule_ingester.consumer.Consumer import Consumer
diff --git a/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py b/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py
new file mode 100644
index 0000000..c28ffbb
--- /dev/null
+++ b/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py
@@ -0,0 +1,71 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 logging
+import os
+import tempfile
+from urllib import parse
+
+import aioboto3
+import xarray as xr
+
+logger = logging.getLogger(__name__)
+
+
+class GranuleLoader:
+
+    def __init__(self, resource: str, *args, **kwargs):
+        super().__init__(*args, **kwargs)
+
+        self._granule_temp_file = None
+        self._resource = resource
+
+    async def __aenter__(self):
+        return await self.open()
+
+    async def __aexit__(self, type, value, traceback):
+        if self._granule_temp_file:
+            self._granule_temp_file.close()
+
+    async def open(self) -> (xr.Dataset, str):
+        resource_url = parse.urlparse(self._resource)
+        if resource_url.scheme == 's3':
+            # We need to save a reference to the temporary granule file so we can delete it when the context manager
+            # closes. The file needs to be kept around until nothing is reading the dataset anymore.
+            self._granule_temp_file = await self._download_s3_file(self._resource)
+            file_path = self._granule_temp_file.name
+        elif resource_url.scheme == '':
+            file_path = self._resource
+        else:
+            raise RuntimeError("Granule path scheme '{}' is not supported.".format(resource_url.scheme))
+
+        granule_name = os.path.basename(self._resource)
+        return xr.open_dataset(file_path, lock=False), granule_name
+
+    @staticmethod
+    async def _download_s3_file(url: str):
+        parsed_url = parse.urlparse(url)
+        logger.info(
+            "Downloading S3 file from bucket '{}' with key '{}'".format(parsed_url.hostname, parsed_url.path[1:]))
+        async with aioboto3.resource("s3") as s3:
+            obj = await s3.Object(bucket_name=parsed_url.hostname, key=parsed_url.path[1:])
+            response = await obj.get()
+            data = await response['Body'].read()
+            logger.info("Finished downloading S3 file.")
+
+        fp = tempfile.NamedTemporaryFile()
+        fp.write(data)
+        logger.info("Saved downloaded file to {}.".format(fp.name))
+        return fp
diff --git a/granule_ingester/granule_ingester/granule_loaders/__init__.py b/granule_ingester/granule_ingester/granule_loaders/__init__.py
new file mode 100644
index 0000000..5df1cb0
--- /dev/null
+++ b/granule_ingester/granule_ingester/granule_loaders/__init__.py
@@ -0,0 +1 @@
+from granule_ingester.granule_loaders.GranuleLoader import GranuleLoader
diff --git a/granule_ingester/granule_ingester/healthcheck/HealthCheck.py b/granule_ingester/granule_ingester/healthcheck/HealthCheck.py
new file mode 100644
index 0000000..390c573
--- /dev/null
+++ b/granule_ingester/granule_ingester/healthcheck/HealthCheck.py
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 abc import ABC, abstractmethod
+
+
+class HealthCheck(ABC):
+    @abstractmethod
+    async def health_check(self) -> bool:
+        pass
diff --git a/granule_ingester/granule_ingester/healthcheck/__init__.py b/granule_ingester/granule_ingester/healthcheck/__init__.py
new file mode 100644
index 0000000..f343c01
--- /dev/null
+++ b/granule_ingester/granule_ingester/healthcheck/__init__.py
@@ -0,0 +1 @@
+from granule_ingester.healthcheck.HealthCheck import HealthCheck
diff --git a/granule_ingester/granule_ingester/main.py b/granule_ingester/granule_ingester/main.py
new file mode 100644
index 0000000..29795f7
--- /dev/null
+++ b/granule_ingester/granule_ingester/main.py
@@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 argparse
+import asyncio
+import logging
+from functools import partial
+from typing import List
+
+from granule_ingester.consumer import Consumer
+from granule_ingester.healthcheck import HealthCheck
+from granule_ingester.writers import CassandraStore
+from granule_ingester.writers import SolrStore
+
+
+def cassandra_factory(contact_points, port):
+    store = CassandraStore(contact_points, port)
+    store.connect()
+    return store
+
+
+def solr_factory(solr_host_and_port):
+    store = SolrStore(solr_host_and_port)
+    store.connect()
+    return store
+
+
+async def run_health_checks(dependencies: List[HealthCheck]):
+    for dependency in dependencies:
+        if not await dependency.health_check():
+            return False
+    return True
+
+
+async def main():
+    parser = argparse.ArgumentParser(description='Process some integers.')
+    parser.add_argument('--rabbitmq_host',
+                        default='localhost',
+                        metavar='HOST',
+                        help='RabbitMQ hostname to connect to. (Default: "localhost")')
+    parser.add_argument('--rabbitmq_username',
+                        default='guest',
+                        metavar='USERNAME',
+                        help='RabbitMQ username. (Default: "guest")')
+    parser.add_argument('--rabbitmq_password',
+                        default='guest',
+                        metavar='PASSWORD',
+                        help='RabbitMQ password. (Default: "guest")')
+    parser.add_argument('--rabbitmq_queue',
+                        default="nexus",
+                        metavar="QUEUE",
+                        help='Name of the RabbitMQ queue to consume from. (Default: "nexus")')
+    parser.add_argument('--cassandra_contact_points',
+                        default=['localhost'],
+                        metavar="HOST",
+                        nargs='+',
+                        help='List of one or more Cassandra contact points, separated by spaces. (Default: "localhost")')
+    parser.add_argument('--cassandra_port',
+                        default=9042,
+                        metavar="PORT",
+                        help='Cassandra port. (Default: 9042)')
+    parser.add_argument('--solr_host_and_port',
+                        default='http://localhost:8983',
+                        metavar='HOST:PORT',
+                        help='Solr host and port. (Default: http://localhost:8983)')
+    parser.add_argument('-v',
+                        '--verbose',
+                        action='store_true',
+                        help='Print verbose logs.')
+
+    args = parser.parse_args()
+
+    logging_level = logging.DEBUG if args.verbose else logging.INFO
+    logging.basicConfig(level=logging_level)
+    loggers = [logging.getLogger(name) for name in logging.root.manager.loggerDict]
+    for logger in loggers:
+        logger.setLevel(logging_level)
+
+    logger = logging.getLogger(__name__)
+
+    config_values_str = "\n".join(["{} = {}".format(arg, getattr(args, arg)) for arg in vars(args)])
+    logger.info("Using configuration values:\n{}".format(config_values_str))
+
+    cassandra_contact_points = args.cassandra_contact_points
+    cassandra_port = args.cassandra_port
+    solr_host_and_port = args.solr_host_and_port
+
+    consumer = Consumer(rabbitmq_host=args.rabbitmq_host,
+                        rabbitmq_username=args.rabbitmq_username,
+                        rabbitmq_password=args.rabbitmq_password,
+                        rabbitmq_queue=args.rabbitmq_queue,
+                        data_store_factory=partial(cassandra_factory, cassandra_contact_points, cassandra_port),
+                        metadata_store_factory=partial(solr_factory, solr_host_and_port))
+    if await run_health_checks(
+            [CassandraStore(cassandra_contact_points, cassandra_port),
+             SolrStore(solr_host_and_port),
+             consumer]):
+        async with consumer:
+            logger.info("All external dependencies have passed the health checks. Now listening to message queue.")
+            await consumer.start_consuming()
+    else:
+        logger.error("Quitting because not all dependencies passed the health checks.")
+
+
+if __name__ == '__main__':
+    asyncio.run(main())
diff --git a/granule_ingester/granule_ingester/pipeline/Modules.py b/granule_ingester/granule_ingester/pipeline/Modules.py
new file mode 100644
index 0000000..2cf2245
--- /dev/null
+++ b/granule_ingester/granule_ingester/pipeline/Modules.py
@@ -0,0 +1,15 @@
+from granule_ingester.processors import *
+from granule_ingester.processors.reading_processors import *
+from granule_ingester.slicers import *
+from granule_ingester.granule_loaders import *
+
+modules = {
+    "granule": GranuleLoader,
+    "sliceFileByStepSize": SliceFileByStepSize,
+    "generateTileId": GenerateTileId,
+    "EccoReadingProcessor": EccoReadingProcessor,
+    "GridReadingProcessor": GridReadingProcessor,
+    "tileSummary": TileSummarizingProcessor,
+    "emptyTileFilter": EmptyTileFilter,
+    "kelvinToCelsius": KelvinToCelsius
+}
diff --git a/granule_ingester/granule_ingester/pipeline/Pipeline.py b/granule_ingester/granule_ingester/pipeline/Pipeline.py
new file mode 100644
index 0000000..8f2dd6f
--- /dev/null
+++ b/granule_ingester/granule_ingester/pipeline/Pipeline.py
@@ -0,0 +1,158 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 logging
+import time
+from typing import List
+
+import aiomultiprocess
+import xarray as xr
+import yaml
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.granule_loaders import GranuleLoader
+from granule_ingester.pipeline.Modules import modules as processor_module_mappings
+from granule_ingester.processors.TileProcessor import TileProcessor
+from granule_ingester.slicers import TileSlicer
+from granule_ingester.writers import DataStore, MetadataStore
+
+logger = logging.getLogger(__name__)
+
+MAX_QUEUE_SIZE = 2 ** 15 - 1
+
+_worker_data_store: DataStore = None
+_worker_metadata_store: MetadataStore = None
+_worker_processor_list: List[TileProcessor] = None
+_worker_dataset = None
+
+
+def _init_worker(processor_list, dataset, data_store_factory, metadata_store_factory):
+    global _worker_data_store
+    global _worker_metadata_store
+    global _worker_processor_list
+    global _worker_dataset
+
+    # _worker_data_store and _worker_metadata_store open multiple TCP sockets from each worker process;
+    # however, these sockets will be automatically closed by the OS once the worker processes die so no need to worry.
+    _worker_data_store = data_store_factory()
+    _worker_metadata_store = metadata_store_factory()
+    _worker_processor_list = processor_list
+    _worker_dataset = dataset
+
+
+async def _process_tile_in_worker(serialized_input_tile: str):
+    global _worker_data_store
+    global _worker_metadata_store
+    global _worker_processor_list
+    global _worker_dataset
+
+    input_tile = nexusproto.NexusTile.FromString(serialized_input_tile)
+    processed_tile = _recurse(_worker_processor_list, _worker_dataset, input_tile)
+    if processed_tile:
+        await _worker_data_store.save_data(processed_tile)
+        await _worker_metadata_store.save_metadata(processed_tile)
+
+
+def _recurse(processor_list: List[TileProcessor],
+             dataset: xr.Dataset,
+             input_tile: nexusproto.NexusTile) -> nexusproto.NexusTile:
+    if len(processor_list) == 0:
+        return input_tile
+    output_tile = processor_list[0].process(tile=input_tile, dataset=dataset)
+    return _recurse(processor_list[1:], dataset, output_tile) if output_tile else None
+
+
+class Pipeline:
+    def __init__(self,
+                 granule_loader: GranuleLoader,
+                 slicer: TileSlicer,
+                 data_store_factory,
+                 metadata_store_factory,
+                 tile_processors: List[TileProcessor]):
+        self._granule_loader = granule_loader
+        self._tile_processors = tile_processors
+        self._slicer = slicer
+        self._data_store_factory = data_store_factory
+        self._metadata_store_factory = metadata_store_factory
+
+    @classmethod
+    def from_string(cls, config_str: str, data_store_factory, metadata_store_factory):
+        config = yaml.load(config_str, yaml.FullLoader)
+        return cls._build_pipeline(config,
+                                   data_store_factory,
+                                   metadata_store_factory,
+                                   processor_module_mappings)
+
+    @classmethod
+    def from_file(cls, config_path: str, data_store_factory, metadata_store_factory):
+        with open(config_path) as config_file:
+            config = yaml.load(config_file, yaml.FullLoader)
+            return cls._build_pipeline(config,
+                                       data_store_factory,
+                                       metadata_store_factory,
+                                       processor_module_mappings)
+
+    @classmethod
+    def _build_pipeline(cls,
+                        config: dict,
+                        data_store_factory,
+                        metadata_store_factory,
+                        module_mappings: dict):
+        granule_loader = GranuleLoader(**config['granule'])
+
+        slicer_config = config['slicer']
+        slicer = cls._parse_module(slicer_config, module_mappings)
+
+        tile_processors = []
+        for processor_config in config['processors']:
+            module = cls._parse_module(processor_config, module_mappings)
+            tile_processors.append(module)
+
+        return cls(granule_loader, slicer, data_store_factory, metadata_store_factory, tile_processors)
+
+    @classmethod
+    def _parse_module(cls, module_config: dict, module_mappings: dict):
+        module_name = module_config.pop('name')
+        try:
+            module_class = module_mappings[module_name]
+            logger.debug("Loaded processor {}.".format(module_class))
+            processor_module = module_class(**module_config)
+        except KeyError:
+            raise RuntimeError("'{}' is not a valid processor.".format(module_name))
+
+        return processor_module
+
+    async def run(self):
+        async with self._granule_loader as (dataset, granule_name):
+            start = time.perf_counter()
+            async with aiomultiprocess.Pool(initializer=_init_worker,
+                                            initargs=(self._tile_processors,
+                                                      dataset,
+                                                      self._data_store_factory,
+                                                      self._metadata_store_factory)) as pool:
+                serialized_tiles = [nexusproto.NexusTile.SerializeToString(tile) for tile in
+                                    self._slicer.generate_tiles(dataset, granule_name)]
+                # aiomultiprocess is built on top of the stdlib multiprocessing library, which has the limitation that
+                # a queue can't have more than 2**15-1 tasks. So, we have to batch it.
+                for chunk in type(self)._chunk_list(serialized_tiles, MAX_QUEUE_SIZE):
+                    await pool.map(_process_tile_in_worker, chunk)
+
+        end = time.perf_counter()
+        logger.info("Pipeline finished in {} seconds".format(end - start))
+
+    @staticmethod
+    def _chunk_list(items, chunk_size):
+        return [items[i:i + chunk_size] for i in range(0, len(items), chunk_size)]
diff --git a/granule_ingester/granule_ingester/pipeline/__init__.py b/granule_ingester/granule_ingester/pipeline/__init__.py
new file mode 100644
index 0000000..7346aa7
--- /dev/null
+++ b/granule_ingester/granule_ingester/pipeline/__init__.py
@@ -0,0 +1,2 @@
+from granule_ingester.pipeline.Pipeline import Pipeline
+from granule_ingester.pipeline.Modules import modules
diff --git a/granule_ingester/granule_ingester/processors/EmptyTileFilter.py b/granule_ingester/granule_ingester/processors/EmptyTileFilter.py
new file mode 100644
index 0000000..4f012f5
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/EmptyTileFilter.py
@@ -0,0 +1,42 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 logging
+
+import numpy
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import from_shaped_array
+
+from granule_ingester.processors.TileProcessor import TileProcessor
+
+logger = logging.getLogger(__name__)
+
+
+def parse_input(nexus_tile_data):
+    return nexusproto.NexusTile.FromString(nexus_tile_data)
+
+
+class EmptyTileFilter(TileProcessor):
+    def process(self, tile, *args, **kwargs):
+        tile_type = tile.tile.WhichOneof("tile_type")
+        tile_data = getattr(tile.tile, tile_type)
+        data = from_shaped_array(tile_data.variable_data)
+
+        # Only supply data if there is actual values in the tile
+        if data.size - numpy.count_nonzero(numpy.isnan(data)) > 0:
+            return tile
+        else:
+            logger.warning("Discarding tile from {} because it is empty".format(tile.summary.granule))
+        return None
diff --git a/granule_ingester/granule_ingester/processors/GenerateTileId.py b/granule_ingester/granule_ingester/processors/GenerateTileId.py
new file mode 100644
index 0000000..2d965f7
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/GenerateTileId.py
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import os
+import uuid
+
+from nexusproto import DataTile_pb2 as nexusproto
+from granule_ingester.processors.TileProcessor import TileProcessor
+
+
+class GenerateTileId(TileProcessor):
+
+    def process(self, tile: nexusproto.NexusTile, *args, **kwargs):
+        granule = os.path.basename(tile.summary.granule)
+        variable_name = tile.summary.data_var_name
+        spec = tile.summary.section_spec
+        generated_id = uuid.uuid3(uuid.NAMESPACE_DNS, granule + variable_name + spec)
+
+        tile.summary.tile_id = str(generated_id)
+        return tile
diff --git a/granule_ingester/granule_ingester/processors/TileProcessor.py b/granule_ingester/granule_ingester/processors/TileProcessor.py
new file mode 100644
index 0000000..d62c504
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/TileProcessor.py
@@ -0,0 +1,23 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 abc import ABC, abstractmethod
+
+
+# TODO: make this an informal interface, not an abstract class
+class TileProcessor(ABC):
+    @abstractmethod
+    def process(self, tile, *args, **kwargs):
+        pass
diff --git a/granule_ingester/granule_ingester/processors/TileSummarizingProcessor.py b/granule_ingester/granule_ingester/processors/TileSummarizingProcessor.py
new file mode 100644
index 0000000..1fe5d7d
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/TileSummarizingProcessor.py
@@ -0,0 +1,98 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 numpy
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import from_shaped_array
+
+from granule_ingester.processors.TileProcessor import TileProcessor
+
+
+class NoTimeException(Exception):
+    pass
+
+
+def find_time_min_max(tile_data):
+    if tile_data.time:
+        if isinstance(tile_data.time, nexusproto.ShapedArray):
+            time_data = from_shaped_array(tile_data.time)
+            return int(numpy.nanmin(time_data).item()), int(numpy.nanmax(time_data).item())
+        elif isinstance(tile_data.time, int):
+            return tile_data.time, tile_data.time
+
+    raise NoTimeException
+
+
+class TileSummarizingProcessor(TileProcessor):
+
+    def __init__(self, dataset_name: str, *args, **kwargs):
+        super().__init__(*args, **kwargs)
+        self._dataset_name = dataset_name
+
+    def process(self, tile, *args, **kwargs):
+        tile_type = tile.tile.WhichOneof("tile_type")
+        tile_data = getattr(tile.tile, tile_type)
+
+        latitudes = numpy.ma.masked_invalid(from_shaped_array(tile_data.latitude))
+        longitudes = numpy.ma.masked_invalid(from_shaped_array(tile_data.longitude))
+        data = from_shaped_array(tile_data.variable_data)
+
+        tile_summary = tile.summary if tile.HasField("summary") else nexusproto.TileSummary()
+
+        tile_summary.dataset_name = self._dataset_name
+        tile_summary.bbox.lat_min = numpy.nanmin(latitudes).item()
+        tile_summary.bbox.lat_max = numpy.nanmax(latitudes).item()
+        tile_summary.bbox.lon_min = numpy.nanmin(longitudes).item()
+        tile_summary.bbox.lon_max = numpy.nanmax(longitudes).item()
+        tile_summary.stats.min = numpy.nanmin(data).item()
+        tile_summary.stats.max = numpy.nanmax(data).item()
+        tile_summary.stats.count = data.size - numpy.count_nonzero(numpy.isnan(data))
+
+        # In order to accurately calculate the average we need to weight the data based on the cosine of its latitude
+        # This is handled slightly differently for swath vs. grid data
+        if tile_type == 'swath_tile':
+            # For Swath tiles, len(data) == len(latitudes) == len(longitudes).
+            # So we can simply weight each element in the data array
+            tile_summary.stats.mean = type(self).calculate_mean_for_swath_tile(data, latitudes)
+        elif tile_type == 'grid_tile':
+            # Grid tiles need to repeat the weight for every longitude
+            # TODO This assumes data axis' are ordered as latitude x longitude
+            tile_summary.stats.mean = type(self).calculate_mean_for_grid_tile(data, latitudes, longitudes)
+        else:
+            # Default to simple average with no weighting
+            tile_summary.stats.mean = numpy.nanmean(data).item()
+
+        try:
+            min_time, max_time = find_time_min_max(tile_data)
+            tile_summary.stats.min_time = min_time
+            tile_summary.stats.max_time = max_time
+        except NoTimeException:
+            pass
+
+        tile.summary.CopyFrom(tile_summary)
+        return tile
+
+    @staticmethod
+    def calculate_mean_for_grid_tile(variable_data, latitudes, longitudes):
+        flattened_variable_data = numpy.ma.masked_invalid(variable_data).flatten()
+        repeated_latitudes = numpy.repeat(latitudes, len(longitudes))
+        weights = numpy.cos(numpy.radians(repeated_latitudes))
+        return numpy.ma.average(flattened_variable_data, weights=weights).item()
+
+    @staticmethod
+    def calculate_mean_for_swath_tile(variable_data, latitudes):
+        weights = numpy.cos(numpy.radians(latitudes))
+        return numpy.ma.average(numpy.ma.masked_invalid(variable_data),
+                                weights=weights).item()
diff --git a/granule_ingester/granule_ingester/processors/__init__.py b/granule_ingester/granule_ingester/processors/__init__.py
new file mode 100644
index 0000000..592d8ea
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/__init__.py
@@ -0,0 +1,5 @@
+from granule_ingester.processors.EmptyTileFilter import EmptyTileFilter
+from granule_ingester.processors.GenerateTileId import GenerateTileId
+from granule_ingester.processors.TileProcessor import TileProcessor
+from granule_ingester.processors.TileSummarizingProcessor import TileSummarizingProcessor
+from granule_ingester.processors.kelvintocelsius import KelvinToCelsius
diff --git a/granule_ingester/granule_ingester/processors/kelvintocelsius.py b/granule_ingester/granule_ingester/processors/kelvintocelsius.py
new file mode 100644
index 0000000..e728418
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/kelvintocelsius.py
@@ -0,0 +1,31 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 nexusproto.serialization import from_shaped_array, to_shaped_array
+
+from granule_ingester.processors.TileProcessor import TileProcessor
+
+
+class KelvinToCelsius(TileProcessor):
+    def process(self, tile, *args, **kwargs):
+        the_tile_type = tile.tile.WhichOneof("tile_type")
+        the_tile_data = getattr(tile.tile, the_tile_type)
+
+        var_data = from_shaped_array(the_tile_data.variable_data) - 273.15
+
+        the_tile_data.variable_data.CopyFrom(to_shaped_array(var_data))
+
+        return tile
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py
new file mode 100644
index 0000000..1876013
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py
@@ -0,0 +1,64 @@
+from typing import Dict
+
+import numpy as np
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import to_shaped_array
+
+from granule_ingester.processors.reading_processors.TileReadingProcessor import TileReadingProcessor
+
+
+class EccoReadingProcessor(TileReadingProcessor):
+    def __init__(self,
+                 variable_to_read,
+                 latitude,
+                 longitude,
+                 tile,
+                 depth=None,
+                 time=None,
+                 **kwargs):
+        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+
+        self.depth = depth
+        self.time = time
+        self.tile = tile
+
+    def _generate_tile(self, ds: xr.Dataset, dimensions_to_slices: Dict[str, slice], input_tile):
+        new_tile = nexusproto.EccoTile()
+
+        lat_subset = ds[self.latitude][type(self)._slices_for_variable(ds[self.latitude], dimensions_to_slices)]
+        lon_subset = ds[self.longitude][type(self)._slices_for_variable(ds[self.longitude], dimensions_to_slices)]
+        lat_subset = np.ma.filled(np.squeeze(lat_subset), np.NaN)
+        lon_subset = np.ma.filled(np.squeeze(lon_subset), np.NaN)
+
+        data_subset = ds[self.variable_to_read][
+            type(self)._slices_for_variable(ds[self.variable_to_read], dimensions_to_slices)]
+        data_subset = np.ma.filled(np.squeeze(data_subset), np.NaN)
+
+        new_tile.tile = ds[self.tile][dimensions_to_slices[self.tile].start].item()
+
+        if self.depth:
+            depth_dim, depth_slice = list(type(self)._slices_for_variable(ds[self.depth],
+                                                                          dimensions_to_slices).items())[0]
+            depth_slice_len = depth_slice.stop - depth_slice.start
+            if depth_slice_len > 1:
+                raise RuntimeError(
+                    "Depth slices must have length 1, but '{dim}' has length {dim_len}.".format(dim=depth_dim,
+                                                                                                dim_len=depth_slice_len))
+            new_tile.depth = ds[self.depth][depth_slice].item()
+
+        if self.time:
+            time_slice = dimensions_to_slices[self.time]
+            time_slice_len = time_slice.stop - time_slice.start
+            if time_slice_len > 1:
+                raise RuntimeError(
+                    "Time slices must have length 1, but '{dim}' has length {dim_len}.".format(dim=self.time,
+                                                                                               dim_len=time_slice_len))
+            new_tile.time = int(ds[self.time][time_slice.start].item() / 1e9)
+
+        new_tile.latitude.CopyFrom(to_shaped_array(lat_subset))
+        new_tile.longitude.CopyFrom(to_shaped_array(lon_subset))
+        new_tile.variable_data.CopyFrom(to_shaped_array(data_subset))
+
+        input_tile.tile.ecco_tile.CopyFrom(new_tile)
+        return input_tile
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py
new file mode 100644
index 0000000..4354f9e
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py
@@ -0,0 +1,53 @@
+from typing import Dict
+
+import numpy as np
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import to_shaped_array
+
+from granule_ingester.processors.reading_processors.TileReadingProcessor import TileReadingProcessor
+
+
+class GridReadingProcessor(TileReadingProcessor):
+    def __init__(self, variable_to_read, latitude, longitude, depth=None, time=None, **kwargs):
+        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+        self.depth = depth
+        self.time = time
+
+    def _generate_tile(self, ds: xr.Dataset, dimensions_to_slices: Dict[str, slice], input_tile):
+        new_tile = nexusproto.GridTile()
+
+        lat_subset = ds[self.latitude][type(self)._slices_for_variable(ds[self.latitude], dimensions_to_slices)]
+        lon_subset = ds[self.longitude][type(self)._slices_for_variable(ds[self.longitude], dimensions_to_slices)]
+        lat_subset = np.ma.filled(np.squeeze(lat_subset), np.NaN)
+        lon_subset = np.ma.filled(np.squeeze(lon_subset), np.NaN)
+
+        data_subset = ds[self.variable_to_read][type(self)._slices_for_variable(ds[self.variable_to_read],
+                                                                                dimensions_to_slices)]
+        data_subset = np.ma.filled(np.squeeze(data_subset), np.NaN)
+
+        if self.depth:
+            depth_dim, depth_slice = list(type(self)._slices_for_variable(ds[self.depth],
+                                                                          dimensions_to_slices).items())[0]
+            depth_slice_len = depth_slice.stop - depth_slice.start
+            if depth_slice_len > 1:
+                raise RuntimeError(
+                    "Depth slices must have length 1, but '{dim}' has length {dim_len}.".format(dim=depth_dim,
+                                                                                                dim_len=depth_slice_len))
+            new_tile.depth = ds[self.depth][depth_slice].item()
+
+        if self.time:
+            time_slice = dimensions_to_slices[self.time]
+            time_slice_len = time_slice.stop - time_slice.start
+            if time_slice_len > 1:
+                raise RuntimeError(
+                    "Time slices must have length 1, but '{dim}' has length {dim_len}.".format(dim=self.time,
+                                                                                               dim_len=time_slice_len))
+            new_tile.time = int(ds[self.time][time_slice.start].item() / 1e9)
+
+        new_tile.latitude.CopyFrom(to_shaped_array(lat_subset))
+        new_tile.longitude.CopyFrom(to_shaped_array(lon_subset))
+        new_tile.variable_data.CopyFrom(to_shaped_array(data_subset))
+
+        input_tile.tile.grid_tile.CopyFrom(new_tile)
+        return input_tile
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py
new file mode 100644
index 0000000..fec28ca
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py
@@ -0,0 +1,47 @@
+from typing import Dict
+
+import numpy as np
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import to_shaped_array
+
+from granule_ingester.processors.reading_processors.TileReadingProcessor import TileReadingProcessor
+
+
+class SwathReadingProcessor(TileReadingProcessor):
+    def __init__(self, variable_to_read, latitude, longitude, time, depth=None, **kwargs):
+        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+        self.depth = depth
+        self.time = time
+
+    def _generate_tile(self, ds: xr.Dataset, dimensions_to_slices: Dict[str, slice], input_tile):
+        new_tile = nexusproto.SwathTile()
+
+        lat_subset = ds[self.latitude][type(self)._slices_for_variable(ds[self.latitude], dimensions_to_slices)]
+        lon_subset = ds[self.longitude][type(self)._slices_for_variable(ds[self.longitude], dimensions_to_slices)]
+        lat_subset = np.ma.filled(lat_subset, np.NaN)
+        lon_subset = np.ma.filled(lon_subset, np.NaN)
+
+        time_subset = ds[self.time][type(self)._slices_for_variable(ds[self.time], dimensions_to_slices)]
+        time_subset = np.ma.filled(type(self)._convert_to_timestamp(time_subset), np.NaN)
+
+        data_subset = ds[self.variable_to_read][type(self)._slices_for_variable(ds[self.variable_to_read],
+                                                                                dimensions_to_slices)]
+        data_subset = np.ma.filled(data_subset, np.NaN)
+
+        if self.depth:
+            depth_dim, depth_slice = list(type(self)._slices_for_variable(ds[self.depth],
+                                                                          dimensions_to_slices).items())[0]
+            depth_slice_len = depth_slice.stop - depth_slice.start
+            if depth_slice_len > 1:
+                raise RuntimeError(
+                    "Depth slices must have length 1, but '{dim}' has length {dim_len}.".format(dim=depth_dim,
+                                                                                                dim_len=depth_slice_len))
+            new_tile.depth = ds[self.depth][depth_slice].item()
+
+        new_tile.latitude.CopyFrom(to_shaped_array(lat_subset))
+        new_tile.longitude.CopyFrom(to_shaped_array(lon_subset))
+        new_tile.variable_data.CopyFrom(to_shaped_array(data_subset))
+        new_tile.time.CopyFrom(to_shaped_array(time_subset))
+        input_tile.tile.swath_tile.CopyFrom(new_tile)
+        return input_tile
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
new file mode 100644
index 0000000..14a44f5
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
@@ -0,0 +1,81 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 datetime
+from abc import ABC, abstractmethod
+from typing import Dict
+
+import numpy as np
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.processors.TileProcessor import TileProcessor
+
+
+class TileReadingProcessor(TileProcessor, ABC):
+
+    def __init__(self, variable_to_read: str, latitude: str, longitude: str, *args, **kwargs):
+        self.variable_to_read = variable_to_read
+        self.latitude = latitude
+        self.longitude = longitude
+
+        # Common optional properties
+        self.temp_dir = None
+        self.metadata = None
+        # self.temp_dir = self.environ['TEMP_DIR']
+        # self.metadata = self.environ['META']
+
+    def process(self, tile, dataset: xr.Dataset, *args, **kwargs):
+        dimensions_to_slices = type(self)._convert_spec_to_slices(tile.summary.section_spec)
+
+        output_tile = nexusproto.NexusTile()
+        output_tile.CopyFrom(tile)
+        output_tile.summary.data_var_name = self.variable_to_read
+
+        return self._generate_tile(dataset, dimensions_to_slices, output_tile)
+
+    @abstractmethod
+    def _generate_tile(self, dataset: xr.Dataset, dimensions_to_slices: Dict[str, slice], tile):
+        pass
+
+    @classmethod
+    def _parse_input(cls, the_input_tile, temp_dir):
+        specs = the_input_tile.summary.section_spec
+        tile_specifications = cls._convert_spec_to_slices(specs)
+
+        file_path = the_input_tile.summary.granule
+        file_path = file_path[len('file:'):] if file_path.startswith('file:') else file_path
+
+        return tile_specifications, file_path
+
+    @staticmethod
+    def _slices_for_variable(variable: xr.DataArray, dimension_to_slice: Dict[str, slice]) -> Dict[str, slice]:
+        return {dim_name: dimension_to_slice[dim_name] for dim_name in variable.dims}
+
+    @staticmethod
+    def _convert_spec_to_slices(spec):
+        dim_to_slice = {}
+        for dimension in spec.split(','):
+            name, start, stop = dimension.split(':')
+            dim_to_slice[name] = slice(int(start), int(stop))
+
+        return dim_to_slice
+
+    @staticmethod
+    def _convert_to_timestamp(times: xr.DataArray) -> xr.DataArray:
+        if times.dtype == np.float32:
+            return times
+        epoch = np.datetime64(datetime.datetime(1970, 1, 1, 0, 0, 0))
+        return ((times - epoch) / 1e9).astype(int)
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py
new file mode 100644
index 0000000..2831c0c
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py
@@ -0,0 +1,83 @@
+from typing import Dict
+
+import numpy as np
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import to_shaped_array
+
+from granule_ingester.processors.reading_processors.TileReadingProcessor import TileReadingProcessor
+
+
+class TimeSeriesReadingProcessor(TileReadingProcessor):
+    def __init__(self, variable_to_read, latitude, longitude, time, depth=None, **kwargs):
+        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+
+        self.depth = depth
+        self.time = time
+
+    def _generate_tile(self, ds: xr.Dataset, dimensions_to_slices: Dict[str, slice], input_tile):
+        new_tile = nexusproto.TimeSeriesTile()
+
+        lat_subset = ds[self.latitude][type(self)._slices_for_variable(ds[self.latitude], dimensions_to_slices)]
+        lon_subset = ds[self.longitude][type(self)._slices_for_variable(ds[self.longitude], dimensions_to_slices)]
+        lat_subset = np.ma.filled(lat_subset, np.NaN)
+        lon_subset = np.ma.filled(lon_subset, np.NaN)
+
+        data_subset = ds[self.variable_to_read][type(self)._slices_for_variable(ds[self.variable_to_read],
+                                                                                dimensions_to_slices)]
+        data_subset = np.ma.filled(data_subset, np.NaN)
+
+        if self.depth:
+            depth_dim, depth_slice = list(type(self)._slices_for_variable(ds[self.depth],
+                                                                          dimensions_to_slices).items())[0]
+            depth_slice_len = depth_slice.stop - depth_slice.start
+            if depth_slice_len > 1:
+                raise RuntimeError(
+                    "Depth slices must have length 1, but '{dim}' has length {dim_len}.".format(dim=depth_dim,
+                                                                                                dim_len=depth_slice_len))
+            new_tile.depth = ds[self.depth][depth_slice].item()
+
+        time_subset = ds[self.time][type(self)._slices_for_variable(ds[self.time], dimensions_to_slices)]
+        time_subset = np.ma.filled(type(self)._convert_to_timestamp(time_subset), np.NaN)
+
+        new_tile.latitude.CopyFrom(to_shaped_array(lat_subset))
+        new_tile.longitude.CopyFrom(to_shaped_array(lon_subset))
+        new_tile.variable_data.CopyFrom(to_shaped_array(data_subset))
+        new_tile.time.CopyFrom(to_shaped_array(time_subset))
+
+        input_tile.tile.time_series_tile.CopyFrom(new_tile)
+        return input_tile
+
+    # def read_data(self, tile_specifications, file_path, output_tile):
+    #     with xr.decode_cf(xr.open_dataset(file_path, decode_cf=False), decode_times=False) as ds:
+    #         for section_spec, dimtoslice in tile_specifications:
+    #             tile = nexusproto.TimeSeriesTile()
+    #
+    #             instance_dimension = next(
+    #                 iter([dim for dim in ds[self.variable_to_read].dims if dim != self.time]))
+    #
+    #             tile.latitude.CopyFrom(
+    #                 to_shaped_array(np.ma.filled(ds[self.latitude].data[dimtoslice[instance_dimension]], np.NaN)))
+    #
+    #             tile.longitude.CopyFrom(
+    #                 to_shaped_array(
+    #                     np.ma.filled(ds[self.longitude].data[dimtoslice[instance_dimension]], np.NaN)))
+    #
+    #             # Before we read the data we need to make sure the dimensions are in the proper order so we don't
+    #             # have any indexing issues
+    #             ordered_slices = slices_for_variable(ds, self.variable_to_read, dimtoslice)
+    #             # Read data using the ordered slices, replacing masked values with NaN
+    #             data_array = np.ma.filled(ds[self.variable_to_read].data[tuple(ordered_slices.values())], np.NaN)
+    #
+    #             tile.variable_data.CopyFrom(to_shaped_array(data_array))
+    #
+    #             if self.metadata is not None:
+    #                 tile.meta_data.add().CopyFrom(
+    #                     to_metadata(self.metadata, ds[self.metadata].data[tuple(ordered_slices.values())]))
+    #
+    #             tile.time.CopyFrom(
+    #                 to_shaped_array(np.ma.filled(ds[self.time].data[dimtoslice[self.time]], np.NaN)))
+    #
+    #             output_tile.tile.time_series_tile.CopyFrom(tile)
+    #
+    #             yield output_tile
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/__init__.py b/granule_ingester/granule_ingester/processors/reading_processors/__init__.py
new file mode 100644
index 0000000..2fecce9
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/reading_processors/__init__.py
@@ -0,0 +1,5 @@
+from granule_ingester.processors.reading_processors.EccoReadingProcessor import EccoReadingProcessor
+from granule_ingester.processors.reading_processors.GridReadingProcessor import GridReadingProcessor
+from granule_ingester.processors.reading_processors.SwathReadingProcessor import SwathReadingProcessor
+from granule_ingester.processors.reading_processors.TileReadingProcessor import TileReadingProcessor
+from granule_ingester.processors.reading_processors.TimeSeriesReadingProcessor import TimeSeriesReadingProcessor
diff --git a/granule_ingester/granule_ingester/slicers/SliceFileByDimension.py b/granule_ingester/granule_ingester/slicers/SliceFileByDimension.py
new file mode 100644
index 0000000..193cf69
--- /dev/null
+++ b/granule_ingester/granule_ingester/slicers/SliceFileByDimension.py
@@ -0,0 +1,55 @@
+# import math
+# import itertools
+# from typing import List, Dict, Tuple,Set
+#
+# # from granule_ingester.entities import Dimension
+#
+#
+# class SliceFileByDimension:
+#     def __init__(self,
+#                  slice_dimension: str,    # slice by this dimension
+#                  dimension_name_prefix: str = None,
+#                  *args, **kwargs):
+#         super().__init__(*args, **kwargs)
+#         self._slice_by_dimension = slice_dimension
+#         self._dimension_name_prefix = dimension_name_prefix
+#
+#     def generate_slices(self, dimension_specs: Dict[str, int]) -> List[str]:
+#         """
+#         Generate list of slices in all dimensions as strings.
+#
+#         :param dimension_specs: A dict of dimension names to dimension lengths
+#         :return: A list of slices across all dimensions, as strings in the form of dim1:0:720,dim2:0:1,dim3:0:360
+#         """
+#         # check if sliceDimension is int or str
+#         is_integer: bool = False
+#         try:
+#             int(self._slice_by_dimension)
+#             is_integer = True
+#         except ValueError:
+#             pass
+#
+#         return self._indexed_dimension_slicing(dimension_specs) if is_integer else self._generate_tile_boundary_slices(self._slice_by_dimension,dimension_specs)
+#
+#     def _indexed_dimension_slicing(self, dimension_specs):
+#         # python netCDF4 library automatically prepends "phony_dim" if indexed by integer
+#         if self._dimension_name_prefix == None or self._dimension_name_prefix == "":
+#             self._dimension_name_prefix = "phony_dim_"
+#
+#         return self._generate_tile_boundary_slices((self._dimension_name_prefix+self._slice_by_dimension),dimension_specs)
+#
+#     def _generate_tile_boundary_slices(self, slice_by_dimension, dimension_specs):
+#         dimension_bounds = []
+#         for dim_name,dim_len in dimension_specs.items():
+#             step_size = 1 if dim_name==slice_by_dimension else dim_len
+#
+#             bounds = []
+#             for i in range(0,dim_len,step_size):
+#                 bounds.append(
+#                     '{name}:{start}:{end}'.format(name=dim_name,
+#                                     start=i,
+#                                     end=min((i + step_size),dim_len) )
+#                 )
+#             dimension_bounds.append(bounds)
+#         return [','.join(chunks) for chunks in itertools.product(*dimension_bounds)]
+#
diff --git a/granule_ingester/granule_ingester/slicers/SliceFileByStepSize.py b/granule_ingester/granule_ingester/slicers/SliceFileByStepSize.py
new file mode 100644
index 0000000..6e03336
--- /dev/null
+++ b/granule_ingester/granule_ingester/slicers/SliceFileByStepSize.py
@@ -0,0 +1,55 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 itertools
+import logging
+from typing import List, Dict
+
+from granule_ingester.slicers.TileSlicer import TileSlicer
+
+logger = logging.getLogger(__name__)
+
+
+class SliceFileByStepSize(TileSlicer):
+    def __init__(self,
+                 dimension_step_sizes: Dict[str, int],
+                 *args, **kwargs):
+        super().__init__(*args, **kwargs)
+        self._dimension_step_sizes = dimension_step_sizes
+
+    def _generate_slices(self, dimension_specs: Dict[str, int]) -> List[str]:
+        # make sure all provided dimensions are in dataset
+        for dim_name in self._dimension_step_sizes.keys():
+            if dim_name not in list(dimension_specs.keys()):
+                raise KeyError('Provided dimension "{}" not found in dataset'.format(dim_name))
+
+        slices = self._generate_chunk_boundary_slices(dimension_specs)
+        logger.info("Sliced granule into {} slices.".format(len(slices)))
+        return slices
+
+    def _generate_chunk_boundary_slices(self, dimension_specs) -> list:
+        dimension_bounds = []
+        dim_step_keys = self._dimension_step_sizes.keys()
+
+        for dim_name, dim_len in dimension_specs.items():
+            step_size = self._dimension_step_sizes[dim_name] if dim_name in dim_step_keys else dim_len
+
+            bounds = []
+            for i in range(0, dim_len, step_size):
+                bounds.append('{name}:{start}:{end}'.format(name=dim_name,
+                                                            start=i,
+                                                            end=min((i + step_size), dim_len)))
+            dimension_bounds.append(bounds)
+        return [','.join(chunks) for chunks in itertools.product(*dimension_bounds)]
diff --git a/granule_ingester/granule_ingester/slicers/SliceFileByTilesDesired.py b/granule_ingester/granule_ingester/slicers/SliceFileByTilesDesired.py
new file mode 100644
index 0000000..a4d401a
--- /dev/null
+++ b/granule_ingester/granule_ingester/slicers/SliceFileByTilesDesired.py
@@ -0,0 +1,68 @@
+# import math
+# import itertools
+# from typing import List, Dict, Tuple
+#
+# # from granule_ingester.entities import Dimension
+#
+#
+# class SliceFileByTilesDesired:
+#     def __init__(self,
+#                  tiles_desired: int,
+#                  desired_spatial_dimensions: List[str],
+#                  time_dimension: Tuple[str, int] = None,
+#                  *args, **kwargs):
+#         super().__init__(*args, **kwargs)
+#         self._tiles_desired = tiles_desired
+#
+#         # check that desired_dimensions have no duplicates
+#         self._desired_spatial_dimensions = desired_spatial_dimensions
+#         self._time_dimension = time_dimension
+#
+#     def generate_slices(self,
+#                         dimension_specs: Dict[str, int]) -> List[str]:
+#         # check that dimension_specs contain all desired_dimensions
+#         # check that there are no duplicate keys in dimension_specs
+#         desired_dimension_specs = {key: dimension_specs[key]
+#                                    for key in self._desired_spatial_dimensions}
+#         spatial_slices = self._generate_spatial_slices(tiles_desired=self._tiles_desired,
+#                                                        dimension_specs=desired_dimension_specs)
+#         if self._time_dimension:
+#             temporal_slices = self._generate_temporal_slices(self._time_dimension)
+#             return self._add_temporal_slices(temporal_slices, spatial_slices)
+#         return spatial_slices
+#
+#     def _add_temporal_slices(self, temporal_slices, spatial_slices) -> List[str]:
+#         return ['{time},{space}'.format(time=temporal_slice, space=spatial_slice)
+#                 for spatial_slice in spatial_slices
+#                 for temporal_slice in temporal_slices]
+#
+#     def _generate_temporal_slices(self, time_dimension: Tuple[str, int]):
+#         return ['{time_dim}:{start}:{end}'.format(time_dim=time_dimension[0],
+#                                                   start=i,
+#                                                   end=i+1)
+#                 for i in range(time_dimension[1]-1)]
+#
+#     def _generate_spatial_slices(self,
+#                                  tiles_desired: int,
+#                                  dimension_specs: Dict[str, int]) -> List[str]:
+#         n_dimensions = len(dimension_specs)
+#         dimension_bounds = []
+#         for dim_name, dim_length in dimension_specs.items():
+#             step_size = SliceFileByTilesDesired._calculate_step_size(
+#                 dim_length, tiles_desired, n_dimensions)
+#             bounds = []
+#             start_loc = 0
+#             while start_loc < dim_length:
+#                 end_loc = start_loc + step_size if start_loc + \
+#                     step_size < dim_length else dim_length
+#                 bounds.append('{name}:{start}:{end}'.format(name=dim_name,
+#                                                             start=start_loc,
+#                                                             end=end_loc))
+#                 start_loc += step_size
+#             dimension_bounds.append(bounds)
+#         return [','.join(chunks) for chunks in itertools.product(*dimension_bounds)]
+#
+#     @staticmethod
+#     def _calculate_step_size(dim_length, chunks_desired, n_dimensions) -> int:
+#         chunks_per_dim = math.pow(chunks_desired, 1.0 / n_dimensions)
+#         return math.floor(dim_length / chunks_per_dim)
diff --git a/granule_ingester/granule_ingester/slicers/TileSlicer.py b/granule_ingester/granule_ingester/slicers/TileSlicer.py
new file mode 100644
index 0000000..06cf094
--- /dev/null
+++ b/granule_ingester/granule_ingester/slicers/TileSlicer.py
@@ -0,0 +1,56 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 abc import ABC, abstractmethod
+from typing import List
+
+import xarray as xr
+from nexusproto.DataTile_pb2 import NexusTile
+
+
+class TileSlicer(ABC):
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(*args, **kwargs)
+
+        self._granule_name = None
+        self._current_tile_spec_index = 0
+        self._tile_spec_list: List[str] = []
+
+    def __iter__(self):
+        return self
+
+    def __next__(self) -> NexusTile:
+        if self._current_tile_spec_index == len(self._tile_spec_list):
+            raise StopIteration
+
+        current_tile_spec = self._tile_spec_list[self._current_tile_spec_index]
+        self._current_tile_spec_index += 1
+
+        tile = NexusTile()
+        tile.summary.section_spec = current_tile_spec
+        tile.summary.granule = self._granule_name
+        return tile
+
+    def generate_tiles(self, dataset: xr.Dataset, granule_name: str = None):
+        self._granule_name = granule_name
+        dimensions = dataset.dims
+        self._tile_spec_list = self._generate_slices(dimensions)
+
+        return self
+
+    @abstractmethod
+    def _generate_slices(self, dimensions):
+        pass
diff --git a/granule_ingester/granule_ingester/slicers/__init__.py b/granule_ingester/granule_ingester/slicers/__init__.py
new file mode 100644
index 0000000..b13ea59
--- /dev/null
+++ b/granule_ingester/granule_ingester/slicers/__init__.py
@@ -0,0 +1,2 @@
+from granule_ingester.slicers.SliceFileByStepSize import SliceFileByStepSize
+from granule_ingester.slicers.TileSlicer import TileSlicer
diff --git a/granule_ingester/granule_ingester/writers/CassandraStore.py b/granule_ingester/granule_ingester/writers/CassandraStore.py
new file mode 100644
index 0000000..7a9f146
--- /dev/null
+++ b/granule_ingester/granule_ingester/writers/CassandraStore.py
@@ -0,0 +1,78 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 asyncio
+import logging
+import uuid
+
+from cassandra.cluster import Cluster, Session
+from cassandra.cqlengine import columns
+from cassandra.cqlengine.models import Model
+from nexusproto.DataTile_pb2 import NexusTile, TileData
+
+from granule_ingester.writers.DataStore import DataStore
+
+logging.getLogger('cassandra').setLevel(logging.INFO)
+logger = logging.getLogger(__name__)
+
+
+class TileModel(Model):
+    __keyspace__ = "nexustiles"
+    __table_name__ = "sea_surface_temp"
+    tile_id = columns.UUID(primary_key=True)
+    tile_blob = columns.Bytes(index=True)
+
+
+class CassandraStore(DataStore):
+    def __init__(self, contact_points=None, port=9042):
+        self._contact_points = contact_points
+        self._port = port
+        self._session = None
+
+    async def health_check(self) -> bool:
+        try:
+            session = self._get_session()
+            session.shutdown()
+            return True
+        except:
+            logger.error("Cannot connect to Cassandra!")
+            return False
+
+    def _get_session(self) -> Session:
+        cluster = Cluster(contact_points=self._contact_points, port=self._port)
+        session = cluster.connect()
+        session.set_keyspace('nexustiles')
+        return session
+
+    def connect(self):
+        self._session = self._get_session()
+
+    def __del__(self):
+        if self._session:
+            self._session.shutdown()
+
+    async def save_data(self, tile: NexusTile) -> None:
+        tile_id = uuid.UUID(tile.summary.tile_id)
+        serialized_tile_data = TileData.SerializeToString(tile.tile)
+        prepared_query = self._session.prepare("INSERT INTO sea_surface_temp (tile_id, tile_blob) VALUES (?, ?)")
+        await type(self)._execute_query_async(self._session, prepared_query, [tile_id, bytearray(serialized_tile_data)])
+
+    @staticmethod
+    async def _execute_query_async(session: Session, query, parameters=None):
+        cassandra_future = session.execute_async(query, parameters)
+        asyncio_future = asyncio.Future()
+        cassandra_future.add_callbacks(asyncio_future.set_result, asyncio_future.set_exception)
+        return await asyncio_future
diff --git a/granule_ingester/granule_ingester/writers/DataStore.py b/granule_ingester/granule_ingester/writers/DataStore.py
new file mode 100644
index 0000000..889d41e
--- /dev/null
+++ b/granule_ingester/granule_ingester/writers/DataStore.py
@@ -0,0 +1,13 @@
+from abc import ABC, abstractmethod
+
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.healthcheck import HealthCheck
+
+
+class DataStore(HealthCheck, ABC):
+
+    @abstractmethod
+    def save_data(self, nexus_tile: nexusproto.NexusTile) -> None:
+        pass
+
diff --git a/granule_ingester/granule_ingester/writers/MetadataStore.py b/granule_ingester/granule_ingester/writers/MetadataStore.py
new file mode 100644
index 0000000..26311af
--- /dev/null
+++ b/granule_ingester/granule_ingester/writers/MetadataStore.py
@@ -0,0 +1,11 @@
+from abc import ABC, abstractmethod
+
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.healthcheck import HealthCheck
+
+
+class MetadataStore(HealthCheck, ABC):
+    @abstractmethod
+    def save_metadata(self, nexus_tile: nexusproto.NexusTile) -> None:
+        pass
diff --git a/granule_ingester/granule_ingester/writers/SolrStore.py b/granule_ingester/granule_ingester/writers/SolrStore.py
new file mode 100644
index 0000000..9d6a7f0
--- /dev/null
+++ b/granule_ingester/granule_ingester/writers/SolrStore.py
@@ -0,0 +1,152 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 asyncio import AbstractEventLoop
+
+import logging
+from datetime import datetime
+from pathlib import Path
+from typing import Dict
+
+import aiohttp
+from nexusproto.DataTile_pb2 import *
+from tenacity import *
+
+from granule_ingester.writers.MetadataStore import MetadataStore
+
+logger = logging.getLogger(__name__)
+
+
+class SolrStore(MetadataStore):
+    def __init__(self, host_and_port='http://localhost:8983'):
+        super().__init__()
+
+        self.TABLE_NAME = "sea_surface_temp"
+        self.iso: str = '%Y-%m-%dT%H:%M:%SZ'
+
+        self._host_and_port = host_and_port
+        self.geo_precision: int = 3
+        self.collection: str = "nexustiles"
+        self.log: logging.Logger = logging.getLogger(__name__)
+        self.log.setLevel(logging.DEBUG)
+        self._session = None
+
+    def connect(self, loop: AbstractEventLoop = None):
+        self._session = aiohttp.ClientSession(loop=loop)
+
+    async def health_check(self):
+        try:
+            async with aiohttp.ClientSession() as session:
+                response = await session.get('{}/solr/{}/admin/ping'.format(self._host_and_port, self.collection))
+                if response.status == 200:
+                    return True
+                else:
+                    logger.error("Solr health check returned status {}.".format(response.status))
+        except aiohttp.ClientConnectionError as e:
+            logger.error("Cannot connect to Solr!")
+
+        return False
+
+    async def save_metadata(self, nexus_tile: NexusTile) -> None:
+        solr_doc = self._build_solr_doc(nexus_tile)
+
+        await self._save_document(self.collection, solr_doc)
+
+    @retry(stop=stop_after_attempt(5))
+    async def _save_document(self, collection: str, doc: dict):
+        url = '{}/solr/{}/update/json/docs?commit=true'.format(self._host_and_port, collection)
+        response = await self._session.post(url, json=doc)
+        if response.status < 200 or response.status >= 400:
+            raise RuntimeError("Saving data to Solr failed with HTTP status code {}".format(response.status))
+
+    def _build_solr_doc(self, tile: NexusTile) -> Dict:
+        summary: TileSummary = tile.summary
+        bbox: TileSummary.BBox = summary.bbox
+        stats: TileSummary.DataStats = summary.stats
+
+        min_time = datetime.strftime(datetime.utcfromtimestamp(stats.min_time), self.iso)
+        max_time = datetime.strftime(datetime.utcfromtimestamp(stats.max_time), self.iso)
+
+        geo = self.determine_geo(bbox)
+
+        granule_file_name: str = Path(summary.granule).name  # get base filename
+
+        tile_type = tile.tile.WhichOneof("tile_type")
+        tile_data = getattr(tile.tile, tile_type)
+
+        input_document = {
+            'table_s': self.TABLE_NAME,
+            'geo': geo,
+            'id': summary.tile_id,
+            'solr_id_s': '{ds_name}!{tile_id}'.format(ds_name=summary.dataset_name, tile_id=summary.tile_id),
+            'sectionSpec_s': summary.section_spec,
+            'dataset_s': summary.dataset_name,
+            'granule_s': granule_file_name,
+            'tile_var_name_s': summary.data_var_name,
+            'tile_min_lon': bbox.lon_min,
+            'tile_max_lon': bbox.lon_max,
+            'tile_min_lat': bbox.lat_min,
+            'tile_max_lat': bbox.lat_max,
+            'tile_depth': tile_data.depth,
+            'tile_min_time_dt': min_time,
+            'tile_max_time_dt': max_time,
+            'tile_min_val_d': stats.min,
+            'tile_max_val_d': stats.max,
+            'tile_avg_val_d': stats.mean,
+            'tile_count_i': int(stats.count)
+        }
+
+        ecco_tile_id = getattr(tile_data, 'tile', None)
+        if ecco_tile_id:
+            input_document['ecco_tile'] = ecco_tile_id
+
+        for attribute in summary.global_attributes:
+            input_document[attribute.getName()] = attribute.getValues(
+                0) if attribute.getValuesCount() == 1 else attribute.getValuesList()
+
+        return input_document
+
+    @staticmethod
+    def _format_latlon_string(value):
+        rounded_value = round(value, 3)
+        return '{:.3f}'.format(rounded_value)
+
+    @classmethod
+    def determine_geo(cls, bbox: TileSummary.BBox) -> str:
+        # Solr cannot index a POLYGON where all corners are the same point or when there are only
+        # 2 distinct points (line). Solr is configured for a specific precision so we need to round
+        # to that precision before checking equality.
+        lat_min_str = cls._format_latlon_string(bbox.lat_min)
+        lat_max_str = cls._format_latlon_string(bbox.lat_max)
+        lon_min_str = cls._format_latlon_string(bbox.lon_min)
+        lon_max_str = cls._format_latlon_string(bbox.lon_max)
+
+        # If lat min = lat max and lon min = lon max, index the 'geo' bounding box as a POINT instead of a POLYGON
+        if bbox.lat_min == bbox.lat_max and bbox.lon_min == bbox.lon_max:
+            geo = 'POINT({} {})'.format(lon_min_str, lat_min_str)
+        # If lat min = lat max but lon min != lon max, or lon min = lon max but lat min != lat max,
+        # then we essentially have a line.
+        elif bbox.lat_min == bbox.lat_max or bbox.lon_min == bbox.lon_max:
+            geo = 'LINESTRING({} {}, {} {})'.format(lon_min_str, lat_min_str, lon_max_str, lat_min_str)
+        # All other cases should use POLYGON
+        else:
+            geo = 'POLYGON(({} {}, {} {}, {} {}, {} {}, {} {}))'.format(lon_min_str, lat_min_str,
+                                                                        lon_max_str, lat_min_str,
+                                                                        lon_max_str, lat_max_str,
+                                                                        lon_min_str, lat_max_str,
+                                                                        lon_min_str, lat_min_str)
+
+        return geo
diff --git a/granule_ingester/granule_ingester/writers/__init__.py b/granule_ingester/granule_ingester/writers/__init__.py
new file mode 100644
index 0000000..9323d8c
--- /dev/null
+++ b/granule_ingester/granule_ingester/writers/__init__.py
@@ -0,0 +1,4 @@
+from granule_ingester.writers.DataStore import DataStore
+from granule_ingester.writers.MetadataStore import MetadataStore
+from granule_ingester.writers.SolrStore import SolrStore
+from granule_ingester.writers.CassandraStore import CassandraStore
diff --git a/granule_ingester/requirements.txt b/granule_ingester/requirements.txt
new file mode 100644
index 0000000..4d9d4cb
--- /dev/null
+++ b/granule_ingester/requirements.txt
@@ -0,0 +1,3 @@
+cassandra-driver==3.23.0
+aiomultiprocess
+aioboto3
diff --git a/granule_ingester/setup.py b/granule_ingester/setup.py
new file mode 100644
index 0000000..2a5920e
--- /dev/null
+++ b/granule_ingester/setup.py
@@ -0,0 +1,34 @@
+from subprocess import check_call, CalledProcessError
+
+from setuptools import setup, find_packages
+
+with open('requirements.txt') as f:
+    pip_requirements = f.readlines()
+
+try:
+    check_call(['conda', 'install', '-y', '-c', 'conda-forge', '--file', 'conda-requirements.txt'])
+except (CalledProcessError, IOError) as e:
+    raise EnvironmentError("Error installing conda packages", e)
+
+__version__ = '1.0.0-SNAPSHOT'
+
+setup(
+    name='sdap_granule_ingester',
+    version=__version__,
+    url="https://github.com/apache/incubator-sdap-ingester",
+    author="dev@sdap.apache.org",
+    author_email="dev@sdap.apache.org",
+    description="Python modules that can be used for NEXUS ingest.",
+    install_requires=pip_requirements,
+    packages=find_packages(
+        exclude=["*.tests", "*.tests.*", "tests.*", "tests", "scripts"]),
+    test_suite="tests",
+    platforms='any',
+    python_requires='>=3.7',
+    classifiers=[
+        'Development Status :: 1 - Pre-Alpha',
+        'Intended Audience :: Developers',
+        'Operating System :: OS Independent',
+        'Programming Language :: Python :: 3.7',
+    ]
+)
diff --git a/granule_ingester/tests/__init__.py b/granule_ingester/tests/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/tests/config_files/analysed_sst.yml b/granule_ingester/tests/config_files/analysed_sst.yml
new file mode 100644
index 0000000..9148f98
--- /dev/null
+++ b/granule_ingester/tests/config_files/analysed_sst.yml
@@ -0,0 +1,16 @@
+slicer:
+  name: sliceFileByStepSize
+  dimension_step_sizes:
+    time: 1
+    lon: 10
+    lat: 10
+processors:
+  - name: GridReadingProcessor
+    latitude: lat
+    longitude: lon
+    time: time
+    variable_to_read: analysed_sst
+  - name: emptyTileFilter
+  - name: tileSummary
+    dataset_name: AVHRR_sst
+  - name: generateTileId
diff --git a/granule_ingester/tests/config_files/ingestion_config_testfile.yaml b/granule_ingester/tests/config_files/ingestion_config_testfile.yaml
new file mode 100644
index 0000000..9af889d
--- /dev/null
+++ b/granule_ingester/tests/config_files/ingestion_config_testfile.yaml
@@ -0,0 +1,17 @@
+granule:
+  resource: ../foo/bar.nc
+slicer:
+  name: sliceFileByStepSize
+  dimension_step_sizes:
+    time: 1
+    lat: 33
+    lon: 26
+processors:
+  - name: EccoReadingProcessor
+    latitude: YC
+    longitude: XC
+    time: time
+    depth: Z
+    tile: tile
+    variable_to_read: THETA
+  - name: generateTileId
diff --git a/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc b/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc
new file mode 100644
index 0000000..4935c81
Binary files /dev/null and b/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc differ
diff --git a/granule_ingester/tests/granules/OBP_2017_01.nc b/granule_ingester/tests/granules/OBP_2017_01.nc
new file mode 100644
index 0000000..8c9b584
Binary files /dev/null and b/granule_ingester/tests/granules/OBP_2017_01.nc differ
diff --git a/granule_ingester/tests/granules/OBP_native_grid.nc b/granule_ingester/tests/granules/OBP_native_grid.nc
new file mode 100755
index 0000000..addb8a0
Binary files /dev/null and b/granule_ingester/tests/granules/OBP_native_grid.nc differ
diff --git a/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 b/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5
new file mode 100644
index 0000000..11815dd
Binary files /dev/null and b/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 differ
diff --git a/granule_ingester/tests/granules/THETA_199201.nc b/granule_ingester/tests/granules/THETA_199201.nc
new file mode 100644
index 0000000..ad92a61
Binary files /dev/null and b/granule_ingester/tests/granules/THETA_199201.nc differ
diff --git a/granule_ingester/tests/granules/empty_mur.nc4 b/granule_ingester/tests/granules/empty_mur.nc4
new file mode 100644
index 0000000..f65c808
Binary files /dev/null and b/granule_ingester/tests/granules/empty_mur.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_ascatb.nc4 b/granule_ingester/tests/granules/not_empty_ascatb.nc4
new file mode 100644
index 0000000..d8ef90b
Binary files /dev/null and b/granule_ingester/tests/granules/not_empty_ascatb.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_avhrr.nc4 b/granule_ingester/tests/granules/not_empty_avhrr.nc4
new file mode 100644
index 0000000..af24071
Binary files /dev/null and b/granule_ingester/tests/granules/not_empty_avhrr.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_ccmp.nc b/granule_ingester/tests/granules/not_empty_ccmp.nc
new file mode 100644
index 0000000..b7b491d
Binary files /dev/null and b/granule_ingester/tests/granules/not_empty_ccmp.nc differ
diff --git a/granule_ingester/tests/granules/not_empty_mur.nc4 b/granule_ingester/tests/granules/not_empty_mur.nc4
new file mode 100644
index 0000000..09d31fd
Binary files /dev/null and b/granule_ingester/tests/granules/not_empty_mur.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_smap.h5 b/granule_ingester/tests/granules/not_empty_smap.h5
new file mode 100644
index 0000000..956cbc5
Binary files /dev/null and b/granule_ingester/tests/granules/not_empty_smap.h5 differ
diff --git a/granule_ingester/tests/granules/not_empty_wswm.nc b/granule_ingester/tests/granules/not_empty_wswm.nc
new file mode 100644
index 0000000..ce0ebcc
Binary files /dev/null and b/granule_ingester/tests/granules/not_empty_wswm.nc differ
diff --git a/granule_ingester/tests/pipeline/__init__.py b/granule_ingester/tests/pipeline/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/tests/pipeline/test_Pipeline.py b/granule_ingester/tests/pipeline/test_Pipeline.py
new file mode 100644
index 0000000..c18bf8b
--- /dev/null
+++ b/granule_ingester/tests/pipeline/test_Pipeline.py
@@ -0,0 +1,104 @@
+import os
+import unittest
+
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.pipeline.Pipeline import Pipeline
+from granule_ingester.processors import GenerateTileId
+from granule_ingester.processors.reading_processors import EccoReadingProcessor
+from granule_ingester.slicers.SliceFileByStepSize import *
+from granule_ingester.writers import DataStore, MetadataStore
+
+
+class TestPipeline(unittest.TestCase):
+    class MockProcessorNoParams:
+        def __init__(self):
+            pass
+
+    class MockProcessorWithParams:
+        def __init__(self, test_param):
+            self.test_param = test_param
+
+    def test_parse_config(self):
+        class MockDataStore(DataStore):
+            def save_data(self, nexus_tile: nexusproto.NexusTile) -> None:
+                pass
+
+        class MockMetadataStore(MetadataStore):
+            def save_metadata(self, nexus_tile: nexusproto.NexusTile) -> None:
+                pass
+
+        relative_path = "../config_files/ingestion_config_testfile.yaml"
+        file_path = os.path.join(os.path.dirname(__file__), relative_path)
+        pipeline = Pipeline.from_file(config_path=str(file_path),
+                                      data_store_factory=MockDataStore,
+                                      metadata_store_factory=MockMetadataStore)
+
+        self.assertEqual(pipeline._data_store_factory, MockDataStore)
+        self.assertEqual(pipeline._metadata_store_factory, MockMetadataStore)
+        self.assertEqual(type(pipeline._slicer), SliceFileByStepSize)
+        self.assertEqual(type(pipeline._tile_processors[0]), EccoReadingProcessor)
+        self.assertEqual(type(pipeline._tile_processors[1]), GenerateTileId)
+
+    def test_parse_module(self):
+        module_mappings = {
+            "sliceFileByStepSize": SliceFileByStepSize
+        }
+
+        module_config = {
+            "name": "sliceFileByStepSize",
+            "dimension_step_sizes": {
+                "time": 1,
+                "lat": 10,
+                "lon": 10
+            }
+        }
+        module = Pipeline._parse_module(module_config, module_mappings)
+        self.assertEqual(SliceFileByStepSize, type(module))
+        self.assertEqual(module_config['dimension_step_sizes'], module._dimension_step_sizes)
+
+    def test_parse_module_with_no_parameters(self):
+        module_mappings = {"MockModule": TestPipeline.MockProcessorNoParams}
+        module_config = {"name": "MockModule"}
+        module = Pipeline._parse_module(module_config, module_mappings)
+        self.assertEqual(type(module), TestPipeline.MockProcessorNoParams)
+
+    def test_parse_module_with_too_many_parameters(self):
+        module_mappings = {"MockModule": TestPipeline.MockProcessorNoParams}
+        module_config = {
+            "name": "MockModule",
+            "bogus_param": True
+        }
+        self.assertRaises(TypeError, Pipeline._parse_module, module_config, module_mappings)
+
+    def test_parse_module_with_missing_parameters(self):
+        module_mappings = {"MockModule": TestPipeline.MockProcessorWithParams}
+        module_config = {
+            "name": "MockModule"
+        }
+
+        self.assertRaises(TypeError, Pipeline._parse_module, module_config, module_mappings)
+
+    def test_process_tile(self):
+        # class MockIdProcessor:
+        #     def process(self, tile, *args, **kwargs):
+        #         tile.summary.tile_id = "test_id"
+        #         return tile
+        #
+        # class MockReadingProcessor:
+        #     def process(self, tile, *args, **kwargs):
+        #         dataset = kwargs['dataset']
+        #         tile.tile.grid_tile.variable_data.CopyFrom(to_shaped_array(dataset['test_variable']))
+        #         return tile
+        #
+        # test_dataset = xr.Dataset({"test_variable": [1, 2, 3]})
+        # input_tile = nexusproto.NexusTile.SerializeToString(NexusTile())
+        # processor_list = [MockIdProcessor(), MockReadingProcessor()]
+        #
+        # output_tile = _process_tile_in_worker(processor_list, test_dataset, input_tile)
+        # output_tile = nexusproto.NexusTile.FromString(output_tile)
+        # tile_data = from_shaped_array(output_tile.tile.grid_tile.variable_data)
+        #
+        # np.testing.assert_equal(tile_data, [1, 2, 3])
+        # self.assertEqual(output_tile.summary.tile_id, "test_id")
+        ...
diff --git a/granule_ingester/tests/processors/__init__.py b/granule_ingester/tests/processors/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/tests/processors/test_GenerateTileId.py b/granule_ingester/tests/processors/test_GenerateTileId.py
new file mode 100644
index 0000000..17f1677
--- /dev/null
+++ b/granule_ingester/tests/processors/test_GenerateTileId.py
@@ -0,0 +1,22 @@
+import unittest
+
+import uuid
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.processors import GenerateTileId
+
+
+class TestGenerateTileId(unittest.TestCase):
+
+    def test_process(self):
+        processor = GenerateTileId()
+
+        tile = nexusproto.NexusTile()
+        tile.summary.granule = 'test_dir/test_granule.nc'
+        tile.summary.data_var_name = 'test_variable'
+        tile.summary.section_spec = 'i:0:90,j:0:90,k:8:9,nv:0:2,tile:4:5,time:8:9'
+
+        expected_id = uuid.uuid3(uuid.NAMESPACE_DNS,
+                                 'test_granule.nc' + 'test_variable' + 'i:0:90,j:0:90,k:8:9,nv:0:2,tile:4:5,time:8:9')
+
+        self.assertEqual(str(expected_id), processor.process(tile).summary.tile_id)
diff --git a/granule_ingester/tests/reading_processors/__init__.py b/granule_ingester/tests/reading_processors/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py b/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py
new file mode 100644
index 0000000..f2e9f29
--- /dev/null
+++ b/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py
@@ -0,0 +1,64 @@
+import unittest
+from os import path
+
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.processors.reading_processors import EccoReadingProcessor
+
+
+class TestEccoReadingProcessor(unittest.TestCase):
+
+    def test_generate_tile(self):
+        reading_processor = EccoReadingProcessor(variable_to_read='OBP',
+                                                 latitude='YC',
+                                                 longitude='XC',
+                                                 time='time',
+                                                 tile='tile')
+
+        granule_path = path.join(path.dirname(__file__), '../granules/OBP_native_grid.nc')
+        tile_summary = nexusproto.TileSummary()
+        tile_summary.granule = granule_path
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.CopyFrom(tile_summary)
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'tile': slice(10, 11),
+            'j': slice(0, 15),
+            'i': slice(0, 7)
+        }
+        with xr.open_dataset(granule_path, decode_cf=True) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(output_tile.summary.granule, granule_path)
+            self.assertEqual(output_tile.tile.ecco_tile.tile, 10)
+            self.assertEqual(output_tile.tile.ecco_tile.time, 695563200)
+            self.assertEqual(output_tile.tile.ecco_tile.variable_data.shape, [15, 7])
+            self.assertEqual(output_tile.tile.ecco_tile.latitude.shape, [15, 7])
+            self.assertEqual(output_tile.tile.ecco_tile.longitude.shape, [15, 7])
+
+    def test_generate_tile_with_dims_out_of_order(self):
+        reading_processor = EccoReadingProcessor(variable_to_read='OBP',
+                                                 latitude='YC',
+                                                 longitude='XC',
+                                                 time='time',
+                                                 tile='tile')
+        granule_path = path.join(path.dirname(__file__), '../granules/OBP_native_grid.nc')
+        input_tile = nexusproto.NexusTile()
+
+        dimensions_to_slices = {
+            'j': slice(0, 15),
+            'tile': slice(10, 11),
+            'i': slice(0, 7),
+            'time': slice(0, 1)
+        }
+        with xr.open_dataset(granule_path, decode_cf=True) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(output_tile.tile.ecco_tile.tile, 10)
+            self.assertEqual(output_tile.tile.ecco_tile.time, 695563200)
+            self.assertEqual(output_tile.tile.ecco_tile.variable_data.shape, [15, 7])
+            self.assertEqual(output_tile.tile.ecco_tile.latitude.shape, [15, 7])
+            self.assertEqual(output_tile.tile.ecco_tile.longitude.shape, [15, 7])
diff --git a/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py b/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py
new file mode 100644
index 0000000..aec3ae8
--- /dev/null
+++ b/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py
@@ -0,0 +1,265 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 unittest
+from os import path
+
+import numpy as np
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+from nexusproto.serialization import from_shaped_array
+
+from granule_ingester.processors.reading_processors import GridReadingProcessor
+
+
+class TestReadMurData(unittest.TestCase):
+
+    def test_read_empty_mur(self):
+        reading_processor = GridReadingProcessor('analysed_sst', 'lat', 'lon', time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/empty_mur.nc4')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'lat': slice(0, 10),
+            'lon': slice(0, 5)
+        }
+        with xr.open_dataset(granule_path) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual(1451638800, output_tile.tile.grid_tile.time)
+            self.assertEqual([10, 5], output_tile.tile.grid_tile.variable_data.shape)
+            self.assertEqual([10], output_tile.tile.grid_tile.latitude.shape)
+            self.assertEqual([5], output_tile.tile.grid_tile.longitude.shape)
+
+            masked_data = np.ma.masked_invalid(from_shaped_array(output_tile.tile.grid_tile.variable_data))
+            self.assertEqual(0, np.ma.count(masked_data))
+
+    def test_read_not_empty_mur(self):
+        reading_processor = GridReadingProcessor('analysed_sst', 'lat', 'lon', time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/not_empty_mur.nc4')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'lat': slice(0, 10),
+            'lon': slice(0, 5)
+        }
+        with xr.open_dataset(granule_path) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual(1451638800, output_tile.tile.grid_tile.time)
+            self.assertEqual([10, 5], output_tile.tile.grid_tile.variable_data.shape)
+            self.assertEqual([10], output_tile.tile.grid_tile.latitude.shape)
+            self.assertEqual([5], output_tile.tile.grid_tile.longitude.shape)
+
+            masked_data = np.ma.masked_invalid(from_shaped_array(output_tile.tile.grid_tile.variable_data))
+            self.assertEqual(50, np.ma.count(masked_data))
+
+
+class TestReadCcmpData(unittest.TestCase):
+
+    def test_read_not_empty_ccmp(self):
+        reading_processor = GridReadingProcessor('uwnd', 'latitude', 'longitude', time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/not_empty_ccmp.nc')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'latitude': slice(0, 38),
+            'longitude': slice(0, 87)
+        }
+        with xr.open_dataset(granule_path) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual(1451606400, output_tile.tile.grid_tile.time)
+            self.assertEqual([38, 87], output_tile.tile.grid_tile.variable_data.shape)
+            self.assertEqual([38], output_tile.tile.grid_tile.latitude.shape)
+            self.assertEqual([87], output_tile.tile.grid_tile.longitude.shape)
+
+            masked_data = np.ma.masked_invalid(from_shaped_array(output_tile.tile.grid_tile.variable_data))
+            self.assertEqual(3306, np.ma.count(masked_data))
+
+        # test_file = path.join(path.dirname(__file__), 'datafiles', 'not_empty_ccmp.nc')
+        #
+        # ccmp_reader = GridReadingProcessor('uwnd', 'latitude', 'longitude', time='time', meta='vwnd')
+        #
+        # input_tile = nexusproto.NexusTile()
+        # tile_summary = nexusproto.TileSummary()
+        # tile_summary.granule = "file:%s" % test_file
+        # tile_summary.section_spec = "time:0:1,longitude:0:87,latitude:0:38"
+        # input_tile.summary.CopyFrom(tile_summary)
+        #
+        # results = list(ccmp_reader.process(input_tile))
+        #
+        # self.assertEqual(1, len(results))
+        #
+        # # with open('./ccmp_nonempty_nexustile.bin', 'w') as f:
+        # #     f.write(results[0])
+        #
+        # for nexus_tile in results:
+        #     self.assertTrue(nexus_tile.HasField('tile'))
+        #     self.assertTrue(nexus_tile.tile.HasField('grid_tile'))
+        #     self.assertEqual(1, len(nexus_tile.tile.grid_tile.meta_data))
+        #
+        #     tile = nexus_tile.tile.grid_tile
+        #     self.assertEqual(38, from_shaped_array(tile.latitude).size)
+        #     self.assertEqual(87, from_shaped_array(tile.longitude).size)
+        #     self.assertEqual((1, 38, 87), from_shaped_array(tile.variable_data).shape)
+        #
+        # tile1_data = np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.variable_data))
+        # self.assertEqual(3306, np.ma.count(tile1_data))
+        # self.assertAlmostEqual(-78.375,
+        #                        np.ma.min(np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.latitude))),
+        #                        places=3)
+        # self.assertAlmostEqual(-69.125,
+        #                        np.ma.max(np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.latitude))),
+        #                        places=3)
+        #
+        # self.assertEqual(1451606400, results[0].tile.grid_tile.time)
+
+
+class TestReadAvhrrData(unittest.TestCase):
+    def test_read_not_empty_avhrr(self):
+        reading_processor = GridReadingProcessor('analysed_sst', 'lat', 'lon', time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/not_empty_avhrr.nc4')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'lat': slice(0, 5),
+            'lon': slice(0, 10)
+        }
+        with xr.open_dataset(granule_path) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual(1462060800, output_tile.tile.grid_tile.time)
+            self.assertEqual([5, 10], output_tile.tile.grid_tile.variable_data.shape)
+            self.assertEqual([5], output_tile.tile.grid_tile.latitude.shape)
+            self.assertEqual([10], output_tile.tile.grid_tile.longitude.shape)
+
+            masked_data = np.ma.masked_invalid(from_shaped_array(output_tile.tile.grid_tile.variable_data))
+            self.assertEqual(50, np.ma.count(masked_data))
+        # test_file = path.join(path.dirname(__file__), 'datafiles', 'not_empty_avhrr.nc4')
+        #
+        # avhrr_reader = GridReadingProcessor('analysed_sst', 'lat', 'lon', time='time')
+        #
+        # input_tile = nexusproto.NexusTile()
+        # tile_summary = nexusproto.TileSummary()
+        # tile_summary.granule = "file:%s" % test_file
+        # tile_summary.section_spec = "time:0:1,lat:0:10,lon:0:10"
+        # input_tile.summary.CopyFrom(tile_summary)
+        #
+        # results = list(avhrr_reader.process(input_tile))
+        #
+        # self.assertEqual(1, len(results))
+        #
+        # for nexus_tile in results:
+        #     self.assertTrue(nexus_tile.HasField('tile'))
+        #     self.assertTrue(nexus_tile.tile.HasField('grid_tile'))
+        #
+        #     tile = nexus_tile.tile.grid_tile
+        #     self.assertEqual(10, from_shaped_array(tile.latitude).size)
+        #     self.assertEqual(10, from_shaped_array(tile.longitude).size)
+        #     self.assertEqual((1, 10, 10), from_shaped_array(tile.variable_data).shape)
+        #
+        # tile1_data = np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.variable_data))
+        # self.assertEqual(100, np.ma.count(tile1_data))
+        # self.assertAlmostEqual(-39.875,
+        #                        np.ma.min(np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.latitude))),
+        #                        places=3)
+        # self.assertAlmostEqual(-37.625,
+        #                        np.ma.max(np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.latitude))),
+        #                        places=3)
+        #
+        # self.assertEqual(1462060800, results[0].tile.grid_tile.time)
+        # self.assertAlmostEqual(289.71,
+        #                        np.ma.masked_invalid(from_shaped_array(results[0].tile.grid_tile.variable_data))[
+        #                            0, 0, 0],
+        #                        places=3)
+
+
+class TestReadInterpEccoData(unittest.TestCase):
+    def setUp(self):
+        self.module = GridReadingProcessor('OBP', 'latitude', 'longitude', x_dim='i', y_dim='j',
+                                           time='time')
+
+    def test_read_indexed_ecco(self):
+        reading_processor = GridReadingProcessor(variable_to_read='OBP',
+                                                 latitude='latitude',
+                                                 longitude='longitude',
+                                                 time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/OBP_2017_01.nc')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'j': slice(0, 5),
+            'i': slice(0, 10)
+        }
+        with xr.open_dataset(granule_path) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual(1484568000, output_tile.tile.grid_tile.time)
+            self.assertEqual([5, 10], output_tile.tile.grid_tile.variable_data.shape)
+            self.assertEqual([5], output_tile.tile.grid_tile.latitude.shape)
+            self.assertEqual([10], output_tile.tile.grid_tile.longitude.shape)
+
+            masked_data = np.ma.masked_invalid(from_shaped_array(output_tile.tile.grid_tile.variable_data))
+            self.assertEqual(50, np.ma.count(masked_data))
+
+        # test_file = path.join(path.dirname(__file__), 'datafiles', 'OBP_2017_01.nc')
+        #
+        # input_tile = nexusproto.NexusTile()
+        # tile_summary = nexusproto.TileSummary()
+        # tile_summary.granule = "file:%s" % test_file
+        # tile_summary.section_spec = "time:0:1,j:0:10,i:0:10"
+        # input_tile.summary.CopyFrom(tile_summary)
+        #
+        # results = list(self.module.process(input_tile))
+        #
+        # self.assertEqual(1, len(results))
+        #
+        # for nexus_tile in results:
+        #     self.assertTrue(nexus_tile.HasField('tile'))
+        #     self.assertTrue(nexus_tile.tile.HasField('grid_tile'))
+        #
+        #     tile = nexus_tile.tile.grid_tile
+        #     self.assertEqual(10, len(from_shaped_array(tile.latitude)))
+        #     self.assertEqual(10, len(from_shaped_array(tile.longitude)))
+        #
+        #     the_data = np.ma.masked_invalid(from_shaped_array(tile.variable_data))
+        #     self.assertEqual((1, 10, 10), the_data.shape)
+        #     self.assertEqual(100, np.ma.count(the_data))
+        #     self.assertEqual(1484568000, tile.time)
+
+
+if __name__ == '__main__':
+    unittest.main()
diff --git a/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py b/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py
new file mode 100644
index 0000000..55ac4fc
--- /dev/null
+++ b/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py
@@ -0,0 +1,74 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 unittest
+from os import path
+
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.processors.reading_processors import SwathReadingProcessor
+
+
+class TestReadAscatbData(unittest.TestCase):
+    def test_read_not_empty_ascatb(self):
+        reading_processor = SwathReadingProcessor(variable_to_read='wind_speed',
+                                                  latitude='lat',
+                                                  longitude='lon',
+                                                  time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/not_empty_ascatb.nc4')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'NUMROWS': slice(0, 1),
+            'NUMCELLS': slice(0, 82)
+        }
+        with xr.open_dataset(granule_path, decode_cf=True) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual([1, 82], output_tile.tile.swath_tile.time.shape)
+            self.assertEqual([1, 82], output_tile.tile.swath_tile.variable_data.shape)
+            self.assertEqual([1, 82], output_tile.tile.swath_tile.latitude.shape)
+            self.assertEqual([1, 82], output_tile.tile.swath_tile.longitude.shape)
+
+
+class TestReadSmapData(unittest.TestCase):
+    def test_read_not_empty_smap(self):
+        reading_processor = SwathReadingProcessor(
+            variable_to_read='smap_sss',
+            latitude='lat',
+            longitude='lon',
+            time='row_time')
+        granule_path = path.join(path.dirname(__file__), '../granules/not_empty_smap.h5')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'phony_dim_0': slice(0, 38),
+            'phony_dim_1': slice(0, 1)
+        }
+
+        with xr.open_dataset(granule_path, decode_cf=True) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule)
+            self.assertEqual([1], output_tile.tile.swath_tile.time.shape)
+            self.assertEqual([38, 1], output_tile.tile.swath_tile.variable_data.shape)
+            self.assertEqual([38, 1], output_tile.tile.swath_tile.latitude.shape)
+            self.assertEqual([38, 1], output_tile.tile.swath_tile.longitude.shape)
diff --git a/granule_ingester/tests/reading_processors/test_TileReadingProcessor.py b/granule_ingester/tests/reading_processors/test_TileReadingProcessor.py
new file mode 100644
index 0000000..90ae8bb
--- /dev/null
+++ b/granule_ingester/tests/reading_processors/test_TileReadingProcessor.py
@@ -0,0 +1,29 @@
+import unittest
+from collections import OrderedDict
+from os import path
+
+import xarray as xr
+
+from granule_ingester.processors.reading_processors import TileReadingProcessor
+
+
+class TestEccoReadingProcessor(unittest.TestCase):
+
+    def test_slices_for_variable(self):
+        dimensions_to_slices = {
+            'j': slice(0, 1),
+            'tile': slice(0, 1),
+            'i': slice(0, 1),
+            'time': slice(0, 1)
+        }
+
+        expected = {
+            'tile': slice(0, 1, None),
+            'j': slice(0, 1, None),
+            'i': slice(0, 1, None)
+        }
+
+        granule_path = path.join(path.dirname(__file__), '../granules/OBP_native_grid.nc')
+        with xr.open_dataset(granule_path, decode_cf=True) as ds:
+            slices = TileReadingProcessor._slices_for_variable(ds['XC'], dimensions_to_slices)
+            self.assertEqual(slices, expected)
diff --git a/granule_ingester/tests/reading_processors/test_TimeSeriesReadingProcessor.py b/granule_ingester/tests/reading_processors/test_TimeSeriesReadingProcessor.py
new file mode 100644
index 0000000..a936885
--- /dev/null
+++ b/granule_ingester/tests/reading_processors/test_TimeSeriesReadingProcessor.py
@@ -0,0 +1,86 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 unittest
+from os import path
+
+import xarray as xr
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.processors.reading_processors import TimeSeriesReadingProcessor
+
+
+class TestReadWSWMData(unittest.TestCase):
+
+    def test_read_not_empty_wswm(self):
+        reading_processor = TimeSeriesReadingProcessor('Qout', 'lat', 'lon', time='time')
+        granule_path = path.join(path.dirname(__file__), '../granules/not_empty_wswm.nc')
+
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 5832),
+            'rivid': slice(0, 1),
+        }
+        with xr.open_dataset(granule_path) as ds:
+            output_tile = reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+            self.assertEqual(granule_path, output_tile.summary.granule, granule_path)
+            self.assertEqual([5832], output_tile.tile.time_series_tile.time.shape)
+            self.assertEqual([5832, 1], output_tile.tile.time_series_tile.variable_data.shape)
+            self.assertEqual([1], output_tile.tile.time_series_tile.latitude.shape)
+            self.assertEqual([1], output_tile.tile.time_series_tile.longitude.shape)
+
+        # test_file = path.join(path.dirname(__file__), 'datafiles', 'not_empty_wswm.nc')
+        # wswm_reader = TimeSeriesReadingProcessor('Qout', 'lat', 'lon', 'time')
+        #
+        # input_tile = nexusproto.NexusTile()
+        # tile_summary = nexusproto.TileSummary()
+        # tile_summary.granule = "file:%s" % test_file
+        # tile_summary.section_spec = "time:0:5832,rivid:0:1"
+        # input_tile.summary.CopyFrom(tile_summary)
+        #
+        # results = list(wswm_reader.process(input_tile))
+        #
+        # self.assertEqual(1, len(results))
+        #
+        # for nexus_tile in results:
+        #     self.assertTrue(nexus_tile.HasField('tile'))
+        #     self.assertTrue(nexus_tile.tile.HasField('time_series_tile'))
+        #
+        #     tile = nexus_tile.tile.time_series_tile
+        #     self.assertEqual(1, from_shaped_array(tile.latitude).size)
+        #     self.assertEqual(1, from_shaped_array(tile.longitude).size)
+        #     self.assertEqual((5832, 1), from_shaped_array(tile.variable_data).shape)
+        #
+        # tile1_data = np.ma.masked_invalid(from_shaped_array(results[0].tile.time_series_tile.variable_data))
+        # self.assertEqual(5832, np.ma.count(tile1_data))
+        # self.assertAlmostEqual(45.837,
+        #                        np.ma.min(
+        #                            np.ma.masked_invalid(from_shaped_array(results[0].tile.time_series_tile.latitude))),
+        #                        places=3)
+        # self.assertAlmostEqual(-122.789,
+        #                        np.ma.max(
+        #                            np.ma.masked_invalid(from_shaped_array(results[0].tile.time_series_tile.longitude))),
+        #                        places=3)
+        #
+        # tile1_times = from_shaped_array(results[0].tile.time_series_tile.time)
+        # self.assertEqual(852098400, tile1_times[0])
+        # self.assertEqual(915073200, tile1_times[-1])
+        # self.assertAlmostEqual(1.473,
+        #                        np.ma.masked_invalid(from_shaped_array(results[0].tile.time_series_tile.variable_data))[
+        #                            0, 0],
+        #                        places=3)
diff --git a/granule_ingester/tests/slicers/__init__.py b/granule_ingester/tests/slicers/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/tests/slicers/test_SliceFileByDimension.py b/granule_ingester/tests/slicers/test_SliceFileByDimension.py
new file mode 100644
index 0000000..e86442b
--- /dev/null
+++ b/granule_ingester/tests/slicers/test_SliceFileByDimension.py
@@ -0,0 +1,122 @@
+# import unittest
+# from collections import Set
+#
+# from netCDF4 import Dataset
+# from granule_ingester.slicers.SliceFileByDimension import SliceFileByDimension
+#
+#
+# class TestSliceFileByTilesDesired(unittest.TestCase):
+#
+#     def test_generate_slices(self):
+#         netcdf_path = 'tests/granules/THETA_199201.nc'
+#         dataset = Dataset(netcdf_path)
+#         dimension_specs = {value.name: value.size for key,
+#                            value in dataset.dimensions.items()}
+#
+#         slicer = SliceFileByDimension(slice_dimension='depth',
+#                                       dimension_name_prefix=None)
+#         slices = slicer.generate_slices(dimension_specs)
+#         expected_slices = ['nv:0:2,time:0:1,longitude:0:720,depth:0:1,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:1:2,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:2:3,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:3:4,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:4:5,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:5:6,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:6:7,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:7:8,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:8:9,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:9:10,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:10:11,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:11:12,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:12:13,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:13:14,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:14:15,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:15:16,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:16:17,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:17:18,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:18:19,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:19:20,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:20:21,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:21:22,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:22:23,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:23:24,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:24:25,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:25:26,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:26:27,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:27:28,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:28:29,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:29:30,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:30:31,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:31:32,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:32:33,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:33:34,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:34:35,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:35:36,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:36:37,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:37:38,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:38:39,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:39:40,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:40:41,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:41:42,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:42:43,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:43:44,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:44:45,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:45:46,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:46:47,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:47:48,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:48:49,latitude:0:360',
+#                            'nv:0:2,time:0:1,longitude:0:720,depth:49:50,latitude:0:360']
+#
+#         self.assertEqual(slices, expected_slices)
+#
+#     def test_generate_slices_indexed(self):
+#         netcdf_path = 'tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5'
+#         dataset = Dataset(netcdf_path)
+#         dimension_specs = {value.name: value.size for key,
+#                                                       value in dataset.dimensions.items()}
+#
+#         slicer = SliceFileByDimension(slice_dimension='2',
+#                                       dimension_name_prefix='phony_dim_')
+#         slices = slicer.generate_slices(dimension_specs)
+#         expected_slices = [
+#             'phony_dim_0:0:76,phony_dim_1:0:1624,phony_dim_2:0:1',
+#             'phony_dim_0:0:76,phony_dim_1:0:1624,phony_dim_2:1:2',
+#             'phony_dim_0:0:76,phony_dim_1:0:1624,phony_dim_2:2:3',
+#             'phony_dim_0:0:76,phony_dim_1:0:1624,phony_dim_2:3:4'
+#         ]
+#
+#         self.assertEqual(slices, expected_slices)
+#
+#     def test_indexed_dimension_slicing(self):
+#         # for some reason, python automatically prefixes integer-indexed dimensions with "phony_dim_"
+#         dimension_specs = {'phony_dim_0': 8, 'phony_dim_1': 8, 'phony_dim_2': 5}
+#         slicer = SliceFileByDimension(slice_dimension='2',
+#                                       dimension_name_prefix=None)
+#         boundary_slices = slicer._indexed_dimension_slicing(dimension_specs)
+#         expected_slices = [
+#             'phony_dim_0:0:8,phony_dim_1:0:8,phony_dim_2:0:1',
+#             'phony_dim_0:0:8,phony_dim_1:0:8,phony_dim_2:1:2',
+#             'phony_dim_0:0:8,phony_dim_1:0:8,phony_dim_2:2:3',
+#             'phony_dim_0:0:8,phony_dim_1:0:8,phony_dim_2:3:4',
+#             'phony_dim_0:0:8,phony_dim_1:0:8,phony_dim_2:4:5'
+#         ]
+#
+#         self.assertEqual(boundary_slices, expected_slices)
+#
+#     def test_generate_tile_boundary_slices(self):
+#         dimension_specs = {'lat': 8, 'lon': 8, 'depth': 5}
+#         slicer = SliceFileByDimension(slice_dimension='depth',
+#                                       dimension_name_prefix=None)
+#         boundary_slices = slicer._generate_tile_boundary_slices(slicer._slice_by_dimension,dimension_specs)
+#         expected_slices = [
+#             'lat:0:8,lon:0:8,depth:0:1',
+#             'lat:0:8,lon:0:8,depth:1:2',
+#             'lat:0:8,lon:0:8,depth:2:3',
+#             'lat:0:8,lon:0:8,depth:3:4',
+#             'lat:0:8,lon:0:8,depth:4:5'
+#         ]
+#
+#         self.assertEqual(boundary_slices, expected_slices)
+#
+# if __name__ == '__main__':
+#     unittest.main()
diff --git a/granule_ingester/tests/slicers/test_SliceFileByStepSize.py b/granule_ingester/tests/slicers/test_SliceFileByStepSize.py
new file mode 100644
index 0000000..7a8dd51
--- /dev/null
+++ b/granule_ingester/tests/slicers/test_SliceFileByStepSize.py
@@ -0,0 +1,105 @@
+import unittest
+from os import path
+
+import xarray as xr
+
+from granule_ingester.slicers.SliceFileByStepSize import SliceFileByStepSize
+
+
+class TestSliceFileByStepSize(unittest.TestCase):
+
+    def test_generate_slices(self):
+        netcdf_path = path.join(path.dirname(__file__), '../granules/THETA_199201.nc')
+        with xr.open_dataset(netcdf_path, decode_cf=True) as dataset:
+            dimension_steps = {'nv': 2, 'time': 1, 'latitude': 180, 'longitude': 180, 'depth': 2}
+            slicer = SliceFileByStepSize(dimension_step_sizes=dimension_steps)
+            slices = slicer._generate_slices(dimension_specs=dataset.dims)
+            expected_slices = [
+                'depth:0:2,latitude:0:180,longitude:0:180,nv:0:2,time:0:1',
+                'depth:0:2,latitude:0:180,longitude:180:360,nv:0:2,time:0:1',
+                'depth:0:2,latitude:0:180,longitude:360:540,nv:0:2,time:0:1',
+                'depth:0:2,latitude:0:180,longitude:540:720,nv:0:2,time:0:1',
+                'depth:0:2,latitude:180:360,longitude:0:180,nv:0:2,time:0:1',
+                'depth:0:2,latitude:180:360,longitude:180:360,nv:0:2,time:0:1',
+                'depth:0:2,latitude:180:360,longitude:360:540,nv:0:2,time:0:1',
+                'depth:0:2,latitude:180:360,longitude:540:720,nv:0:2,time:0:1',
+                'depth:2:4,latitude:0:180,longitude:0:180,nv:0:2,time:0:1',
+                'depth:2:4,latitude:0:180,longitude:180:360,nv:0:2,time:0:1',
+                'depth:2:4,latitude:0:180,longitude:360:540,nv:0:2,time:0:1',
+                'depth:2:4,latitude:0:180,longitude:540:720,nv:0:2,time:0:1',
+                'depth:2:4,latitude:180:360,longitude:0:180,nv:0:2,time:0:1',
+                'depth:2:4,latitude:180:360,longitude:180:360,nv:0:2,time:0:1',
+                'depth:2:4,latitude:180:360,longitude:360:540,nv:0:2,time:0:1',
+                'depth:2:4,latitude:180:360,longitude:540:720,nv:0:2,time:0:1'
+            ]
+
+            self.assertEqual(expected_slices, slices)
+
+    def test_generate_slices_indexed(self):
+        netcdf_path = path.join(path.dirname(__file__), '../granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5')
+        with xr.open_dataset(netcdf_path, decode_cf=True) as dataset:
+            dimension_steps = {'phony_dim_0': 76, 'phony_dim_1': 812, 'phony_dim_2': 1}
+            slicer = SliceFileByStepSize(dimension_step_sizes=dimension_steps)
+            slices = slicer._generate_slices(dimension_specs=dataset.dims)
+            expected_slices = [
+                'phony_dim_0:0:76,phony_dim_1:0:812,phony_dim_2:0:1',
+                'phony_dim_0:0:76,phony_dim_1:0:812,phony_dim_2:1:2',
+                'phony_dim_0:0:76,phony_dim_1:0:812,phony_dim_2:2:3',
+                'phony_dim_0:0:76,phony_dim_1:0:812,phony_dim_2:3:4',
+                'phony_dim_0:0:76,phony_dim_1:812:1624,phony_dim_2:0:1',
+                'phony_dim_0:0:76,phony_dim_1:812:1624,phony_dim_2:1:2',
+                'phony_dim_0:0:76,phony_dim_1:812:1624,phony_dim_2:2:3',
+                'phony_dim_0:0:76,phony_dim_1:812:1624,phony_dim_2:3:4'
+            ]
+
+            self.assertEqual(slices, expected_slices)
+
+    def test_generate_chunk_boundary_slices(self):
+        dimension_specs = {'time': 5832, 'rivid': 43}
+        dimension_steps = {'time': 2916, 'rivid': 5}
+        slicer = SliceFileByStepSize(dimension_step_sizes=dimension_steps)
+        boundary_slices = slicer._generate_chunk_boundary_slices(dimension_specs)
+        expected_slices = [
+            'time:0:2916,rivid:0:5',
+            'time:0:2916,rivid:5:10',
+            'time:0:2916,rivid:10:15',
+            'time:0:2916,rivid:15:20',
+            'time:0:2916,rivid:20:25',
+            'time:0:2916,rivid:25:30',
+            'time:0:2916,rivid:30:35',
+            'time:0:2916,rivid:35:40',
+            'time:0:2916,rivid:40:43',
+            'time:2916:5832,rivid:0:5',
+            'time:2916:5832,rivid:5:10',
+            'time:2916:5832,rivid:10:15',
+            'time:2916:5832,rivid:15:20',
+            'time:2916:5832,rivid:20:25',
+            'time:2916:5832,rivid:25:30',
+            'time:2916:5832,rivid:30:35',
+            'time:2916:5832,rivid:35:40',
+            'time:2916:5832,rivid:40:43',
+        ]
+
+        self.assertEqual(boundary_slices, expected_slices)
+
+    def test_generate_chunk_boundary_slices_indexed(self):
+        dimension_steps = {'phony_dim_0': 4, 'phony_dim_1': 4, 'phony_dim_2': 3}
+        dimension_specs = {'phony_dim_0': 8, 'phony_dim_1': 8, 'phony_dim_2': 5}
+        slicer = SliceFileByStepSize(dimension_step_sizes=dimension_steps)
+        boundary_slices = slicer._generate_slices(dimension_specs)
+        expected_slices = [
+            'phony_dim_0:0:4,phony_dim_1:0:4,phony_dim_2:0:3',
+            'phony_dim_0:0:4,phony_dim_1:0:4,phony_dim_2:3:5',
+            'phony_dim_0:0:4,phony_dim_1:4:8,phony_dim_2:0:3',
+            'phony_dim_0:0:4,phony_dim_1:4:8,phony_dim_2:3:5',
+            'phony_dim_0:4:8,phony_dim_1:0:4,phony_dim_2:0:3',
+            'phony_dim_0:4:8,phony_dim_1:0:4,phony_dim_2:3:5',
+            'phony_dim_0:4:8,phony_dim_1:4:8,phony_dim_2:0:3',
+            'phony_dim_0:4:8,phony_dim_1:4:8,phony_dim_2:3:5',
+        ]
+
+        self.assertEqual(boundary_slices, expected_slices)
+
+
+if __name__ == '__main__':
+    unittest.main()
diff --git a/granule_ingester/tests/slicers/test_SliceFileByTilesDesired.py b/granule_ingester/tests/slicers/test_SliceFileByTilesDesired.py
new file mode 100644
index 0000000..772c63e
--- /dev/null
+++ b/granule_ingester/tests/slicers/test_SliceFileByTilesDesired.py
@@ -0,0 +1,88 @@
+# import unittest
+# from collections import Set
+#
+# from netCDF4 import Dataset
+# from granule_ingester.slicers.SliceFileByTilesDesired import SliceFileByTilesDesired
+#
+#
+# class TestSliceFileByTilesDesired(unittest.TestCase):
+#
+#     def test_generate_slices(self):
+#         netcdf_path = 'tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc'
+#         dataset = Dataset(netcdf_path)
+#         dimension_specs = {value.name: value.size for key,
+#                            value in dataset.dimensions.items()}
+#
+#         slicer = SliceFileByTilesDesired(tiles_desired=2,
+#                                          desired_spatial_dimensions=['lat', 'lon'])
+#         slices = slicer.generate_slices(dimension_specs)
+#         expected_slices = ['lat:0:509,lon:0:1018',
+#                            'lat:0:509,lon:1018:1440',
+#                            'lat:509:720,lon:0:1018',
+#                            'lat:509:720,lon:1018:1440']
+#         self.assertEqual(slices, expected_slices)
+#
+#     def test_generate_slices_with_time(self):
+#         netcdf_path = 'tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc'
+#         dataset = Dataset(netcdf_path)
+#         dimension_specs = {value.name: value.size for key,
+#                            value in dataset.dimensions.items()}
+#
+#         slicer = SliceFileByTilesDesired(tiles_desired=2,
+#                                          desired_spatial_dimensions=[
+#                                              'lat', 'lon'],
+#                                          time_dimension=('time', 3))
+#         slices = slicer.generate_slices(dimension_specs)
+#         expected_slices = ['time:0:1,lat:0:509,lon:0:1018',
+#                            'time:1:2,lat:0:509,lon:0:1018',
+#
+#                            'time:0:1,lat:0:509,lon:1018:1440',
+#                            'time:1:2,lat:0:509,lon:1018:1440',
+#
+#                            'time:0:1,lat:509:720,lon:0:1018',
+#                            'time:1:2,lat:509:720,lon:0:1018',
+#
+#                            'time:0:1,lat:509:720,lon:1018:1440',
+#                            'time:1:2,lat:509:720,lon:1018:1440']
+#         self.assertEqual(slices, expected_slices)
+#
+#     def test_calculate_step_size_perfect_split_2dim(self):
+#         step_size = SliceFileByTilesDesired._calculate_step_size(1000, 100, 2)
+#         self.assertAlmostEqual(step_size, 100.0)
+#
+#     def test_calculate_step_size_perfect_split_3dim(self):
+#         step_size = SliceFileByTilesDesired._calculate_step_size(1000, 100, 3)
+#         self.assertAlmostEqual(step_size, 215.0)
+#
+#     def test_generate_spatial_slices(self):
+#         dimension_specs = {'lat': 8, 'lon': 8}
+#         slicer = SliceFileByTilesDesired(tiles_desired=2,
+#                                          desired_spatial_dimensions=dimension_specs)
+#         boundary_slices = slicer._generate_spatial_slices(tiles_desired=4,
+#                                                           dimension_specs=dimension_specs)
+#         expected_slices = [
+#             'lat:0:4,lon:0:4',
+#             'lat:0:4,lon:4:8',
+#             'lat:4:8,lon:0:4',
+#             'lat:4:8,lon:4:8'
+#         ]
+#         self.assertEqual(boundary_slices, expected_slices)
+#
+#     def test_generate_temporal_slices(self):
+#         slicer = SliceFileByTilesDesired(tiles_desired=2,
+#                                          desired_spatial_dimensions=None)
+#         time_slices = slicer._generate_temporal_slices(('time', 10))
+#         expected_time_slices = ['time:0:1',
+#                                 'time:1:2',
+#                                 'time:2:3',
+#                                 'time:3:4',
+#                                 'time:4:5',
+#                                 'time:5:6',
+#                                 'time:6:7',
+#                                 'time:7:8',
+#                                 'time:8:9']
+#         self.assertEqual(time_slices, expected_time_slices)
+#
+#
+# if __name__ == '__main__':
+#     unittest.main()
diff --git a/granule_ingester/tests/slicers/test_TileSlicer.py b/granule_ingester/tests/slicers/test_TileSlicer.py
new file mode 100644
index 0000000..c3ad97f
--- /dev/null
+++ b/granule_ingester/tests/slicers/test_TileSlicer.py
@@ -0,0 +1,68 @@
+import asyncio
+import os
+import unittest
+from granule_ingester.slicers.TileSlicer import TileSlicer
+import xarray as xr
+
+
+class TestTileSlicer(unittest.TestCase):
+    class ToyTileSlicer(TileSlicer):
+        def _generate_slices(self, dimensions):
+            return [
+                'time:0:1,lat:0:4,lon:0:4',
+                'time:1:2,lat:0:4,lon:0:4',
+                'time:2:3,lat:0:4,lon:0:4',
+
+                'time:0:1,lat:0:4,lon:4:8',
+                'time:1:2,lat:0:4,lon:4:8',
+                'time:2:3,lat:0:4,lon:4:8',
+
+                'time:0:1,lat:4:8,lon:0:4',
+                'time:1:2,lat:4:8,lon:0:4',
+                'time:2:3,lat:4:8,lon:0:4',
+
+                'time:0:1,lat:4:8,lon:4:8',
+                'time:1:2,lat:4:8,lon:4:8',
+                'time:2:3,lat:4:8,lon:4:8'
+            ]
+
+    def test_generate_tiles(self):
+        relative_path = '../granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc'
+        file_path = os.path.join(os.path.dirname(__file__), relative_path)
+        with xr.open_dataset(file_path) as dataset:
+            slicer = TestTileSlicer.ToyTileSlicer().generate_tiles(dataset, file_path)
+
+        expected_slices = slicer._generate_slices(None)
+        self.assertEqual(file_path, slicer._granule_name)
+        self.assertEqual(expected_slices, slicer._tile_spec_list)
+
+    # def test_open_s3(self):
+    #     s3_path = 's3://nexus-ingest/avhrr/198109-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc'
+    #     slicer = TestTileSlicer.ToyTileSlicer(resource=s3_path)
+    #
+    #     expected_slices = slicer._generate_slices(None)
+    #     asyncio.run(slicer.open())
+    #     self.assertIsNotNone(slicer.dataset)
+    #     self.assertEqual(expected_slices, slicer._tile_spec_list)
+
+    def test_next(self):
+        relative_path = '../granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc'
+        file_path = os.path.join(os.path.dirname(__file__), relative_path)
+        with xr.open_dataset(file_path) as dataset:
+            slicer = TestTileSlicer.ToyTileSlicer().generate_tiles(dataset, file_path)
+        generated_tiles = list(slicer)
+
+        expected_slices = slicer._generate_slices(None)
+        self.assertListEqual(expected_slices, [tile.summary.section_spec for tile in generated_tiles])
+        for tile in generated_tiles:
+            self.assertEqual(file_path, tile.summary.granule)
+
+    # def test_download_s3_file(self):
+    #     slicer = TestTileSlicer.ToyTileSlicer(resource=None)
+    #
+    #     asyncio.run(slicer._download_s3_file(
+    #         "s3://nexus-ingest/avhrr/198109-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc"))
+
+
+if __name__ == '__main__':
+    unittest.main()
diff --git a/granule_ingester/tests/writers/__init__.py b/granule_ingester/tests/writers/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/granule_ingester/tests/writers/test_SolrStore.py b/granule_ingester/tests/writers/test_SolrStore.py
new file mode 100644
index 0000000..76b85ac
--- /dev/null
+++ b/granule_ingester/tests/writers/test_SolrStore.py
@@ -0,0 +1,54 @@
+import asyncio
+import unittest
+
+from nexusproto import DataTile_pb2 as nexusproto
+
+from granule_ingester.writers import SolrStore
+
+
+class TestSolrStore(unittest.TestCase):
+
+    def test_build_solr_doc(self):
+        tile = nexusproto.NexusTile()
+        tile.summary.tile_id = 'test_id'
+        tile.summary.dataset_name = 'test_dataset'
+        tile.summary.dataset_uuid = 'test_dataset_id'
+        tile.summary.data_var_name = 'test_variable'
+        tile.summary.granule = 'test_granule_path'
+        tile.summary.section_spec = 'time:0:1,j:0:20,i:200:240'
+        tile.summary.bbox.lat_min = -180.1
+        tile.summary.bbox.lat_max = 180.2
+        tile.summary.bbox.lon_min = -90.5
+        tile.summary.bbox.lon_max = 90.0
+        tile.summary.stats.min = -10.0
+        tile.summary.stats.max = 25.5
+        tile.summary.stats.mean = 12.5
+        tile.summary.stats.count = 100
+        tile.summary.stats.min_time = 694224000
+        tile.summary.stats.max_time = 694310400
+
+        tile.tile.ecco_tile.depth = 10.5
+
+        metadata_store = SolrStore()
+        solr_doc = metadata_store._build_solr_doc(tile)
+
+        self.assertEqual('sea_surface_temp', solr_doc['table_s'])
+        self.assertEqual(
+            'POLYGON((-90.500 -180.100, 90.000 -180.100, 90.000 180.200, -90.500 180.200, -90.500 -180.100))',
+            solr_doc['geo'])
+        self.assertEqual('test_id', solr_doc['id'])
+        self.assertEqual('test_dataset!test_id', solr_doc['solr_id_s'])
+        self.assertEqual('time:0:1,j:0:20,i:200:240', solr_doc['sectionSpec_s'])
+        self.assertEqual('test_granule_path', solr_doc['granule_s'])
+        self.assertEqual('test_variable', solr_doc['tile_var_name_s'])
+        self.assertAlmostEqual(-90.5, solr_doc['tile_min_lon'])
+        self.assertAlmostEqual(90.0, solr_doc['tile_max_lon'])
+        self.assertAlmostEqual(-180.1, solr_doc['tile_min_lat'])
+        self.assertAlmostEqual(180.2, solr_doc['tile_max_lat'])
+        self.assertEqual('1992-01-01T00:00:00Z', solr_doc['tile_min_time_dt'])
+        self.assertEqual('1992-01-02T00:00:00Z', solr_doc['tile_max_time_dt'])
+        self.assertAlmostEqual(-10.0, solr_doc['tile_min_val_d'])
+        self.assertAlmostEqual(25.5, solr_doc['tile_max_val_d'])
+        self.assertAlmostEqual(12.5, solr_doc['tile_avg_val_d'])
+        self.assertEqual(100, solr_doc['tile_count_i'])
+        self.assertAlmostEqual(10.5, solr_doc['tile_depth'])

[incubator-sdap-ingester] 17/33: SDAP-280: Collection Manager to talk to Ingestion History asynchronously (#16)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 0aa6121320995a1d6c0c7101b86b7d1af15cdabd
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Aug 11 11:45:26 2020 -0700

    SDAP-280: Collection Manager to talk to Ingestion History asynchronously (#16)
---
 collection_manager/README.md                       | 14 ++++----
 .../services/CollectionProcessor.py                |  8 ++---
 .../services/CollectionWatcher.py                  | 15 +++++----
 .../history_manager/FileIngestionHistory.py        |  9 ++---
 .../services/history_manager/IngestionHistory.py   | 26 +++++++--------
 .../history_manager/SolrIngestionHistory.py        | 11 +++----
 collection_manager/docker/Dockerfile               | 12 ++++---
 collection_manager/requirements.txt                |  2 +-
 .../history_manager/test_FileIngestionHistory.py   | 38 +++++++++++++---------
 .../tests/services/test_CollectionProcessor.py     | 16 +++++----
 .../tests/services/test_CollectionWatcher.py       |  1 -
 common/common/async_utils/AsyncUtils.py            | 11 +++++++
 common/common/async_utils/__init__.py              |  1 +
 granule_ingester/README.md                         | 21 ++++++------
 granule_ingester/docker/Dockerfile                 | 14 ++++----
 .../granule_ingester/writers/SolrStore.py          | 17 +++-------
 16 files changed, 118 insertions(+), 98 deletions(-)

diff --git a/collection_manager/README.md b/collection_manager/README.md
index 771f355..84df468 100644
--- a/collection_manager/README.md
+++ b/collection_manager/README.md
@@ -12,15 +12,15 @@ Manager service will publish a message to RabbitMQ to be picked up by the Granul
 Python 3.7
 
 ## Building the service
-From `incubator-sdap-ingester/collection_manager`, run:
-
-    $ python setup.py install
+From `incubator-sdap-ingester`, run:
+    $ cd common && python setup.py install
+    $ cd ../collection_manager python setup.py install
     
 
 ## Running the service
-From `incubator-sdap-ingester/collection_manager`, run:
+From `incubator-sdap-ingester`, run:
 
-    $ python collection_manager/main.py -h
+    $ python collection_manager/collection_manager/main.py -h
     
 ### The Collections Configuration File
 
@@ -71,6 +71,6 @@ From `incubator-sdap-ingester/`, run:
     $ pip install pytest && pytest
     
 ## Building the Docker image
-From `incubator-sdap-ingester/collection_manager`, run:
+From `incubator-sdap-ingester`, run:
 
-    $ docker build . -f docker/Dockerfile -t nexusjpl/collection-manager
+    $ docker build . -f collection_manager/docker/Dockerfile -t nexusjpl/collection-manager
diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index d790f4b..ac61586 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -45,7 +45,7 @@ class CollectionProcessor:
             return
 
         history_manager = self._get_history_manager(collection.dataset_id)
-        granule_status = history_manager.get_granule_status(granule, collection.date_from, collection.date_to)
+        granule_status = await history_manager.get_granule_status(granule, collection.date_from, collection.date_to)
 
         if granule_status is GranuleStatus.DESIRED_FORWARD_PROCESSING:
             logger.info(f"New granule '{granule}' detected for forward-processing ingestion "
@@ -59,13 +59,13 @@ class CollectionProcessor:
                         f"'{collection.dataset_id}'.")
             use_priority = collection.historical_priority
         else:
-            logger.info(f"Granule '{granule}' detected but has already been ingested or is not in desired "
-                        f"time range for collection '{collection.dataset_id}'. Skipping.")
+            logger.debug(f"Granule '{granule}' detected but has already been ingested in "
+                         f"collection '{collection.dataset_id}'. Skipping.")
             return
 
         dataset_config = self._fill_template(granule, collection, config_template=self._config_template)
         await self._publisher.publish_message(body=dataset_config, priority=use_priority)
-        history_manager.push(granule)
+        await history_manager.push(granule)
 
     @staticmethod
     def _file_supported(file_path: str):
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index 8f67e16..54c8877 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -1,18 +1,19 @@
 import asyncio
-import time
 import logging
 import os
+import time
 from collections import defaultdict
-from typing import Dict, Callable, Set, Optional, Awaitable
+from typing import Awaitable, Callable, Dict, Optional, Set
+
 import yaml
+from collection_manager.entities import Collection
+from collection_manager.entities.exceptions import (
+    CollectionConfigFileNotFoundError, CollectionConfigParsingError,
+    ConflictingPathCollectionError, MissingValueCollectionError,
+    RelativePathCollectionError, RelativePathError)
 from watchdog.events import FileSystemEventHandler
 from watchdog.observers.polling import PollingObserver as Observer
 
-from collection_manager.entities import Collection
-from collection_manager.entities.exceptions import RelativePathError, CollectionConfigParsingError, \
-    CollectionConfigFileNotFoundError, MissingValueCollectionError, ConflictingPathCollectionError, \
-    RelativePathCollectionError
-
 logger = logging.getLogger(__name__)
 logger.setLevel(logging.DEBUG)
 
diff --git a/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
index 50f2170..ffa065f 100644
--- a/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
@@ -28,7 +28,8 @@ class FileIngestionHistory(IngestionHistory):
         Constructor
         :param history_path:
         :param dataset_id:
-        :param signature_fun: function which create the signature of the cache, a file path string as argument and returns a string (md5sum, time stamp)
+        :param signature_fun: function which creates the signature of the cache,
+                              a file path string as argument and returns a string (md5sum, time stamp)
         """
         self._dataset_id = dataset_id
         self._history_file_path = os.path.join(history_path, f'{dataset_id}.csv')
@@ -65,7 +66,7 @@ class FileIngestionHistory(IngestionHistory):
         except FileNotFoundError:
             logger.info(f"history cache {self._history_file_path} does not exist, does not need to be removed")
 
-    def _save_latest_timestamp(self):
+    async def _save_latest_timestamp(self):
         if self._latest_ingested_file_update:
             with open(self._latest_ingested_file_update_file_path, 'w') as f_ts:
                 f_ts.write(f'{str(self._latest_ingested_file_update)}\n')
@@ -89,10 +90,10 @@ class FileIngestionHistory(IngestionHistory):
         except FileNotFoundError:
             logger.info(f"no history file {self._history_file_path} to purge")
 
-    def _push_record(self, file_name, signature):
+    async def _push_record(self, file_name, signature):
         self._history_dict[file_name] = signature
 
         self._history_file.write(f'{file_name},{signature}\n')
 
-    def _get_signature(self, file_name):
+    async def _get_signature(self, file_name):
         return self._history_dict.get(file_name, None)
diff --git a/collection_manager/collection_manager/services/history_manager/IngestionHistory.py b/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
index d92cb24..ef73ccb 100644
--- a/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
@@ -37,7 +37,7 @@ class IngestionHistory(ABC):
     _signature_fun = None
     _latest_ingested_file_update = None
 
-    def push(self, file_path: str):
+    async def push(self, file_path: str):
         """
         Record a file as having been ingested.
         :param file_path: The full path to the file to record.
@@ -46,14 +46,14 @@ class IngestionHistory(ABC):
         file_path = file_path.strip()
         file_name = os.path.basename(file_path)
         signature = self._signature_fun(file_path)
-        self._push_record(file_name, signature)
+        await self._push_record(file_name, signature)
 
         if not self._latest_ingested_file_update:
             self._latest_ingested_file_update = os.path.getmtime(file_path)
         else:
             self._latest_ingested_file_update = max(self._latest_ingested_file_update, os.path.getmtime(file_path))
 
-        self._save_latest_timestamp()
+        await self._save_latest_timestamp()
 
     def latest_ingested_mtime(self) -> Optional[datetime]:
         """
@@ -65,7 +65,7 @@ class IngestionHistory(ABC):
         else:
             return None
 
-    def already_ingested(self, file_path: str) -> bool:
+    async def already_ingested(self, file_path: str) -> bool:
         """
         Return a boolean indicating whether the specified file has already been ingested, based on its signature.
         :param file_path: The full path of a file to search for in the history.
@@ -74,12 +74,12 @@ class IngestionHistory(ABC):
         file_path = file_path.strip()
         file_name = os.path.basename(file_path)
         signature = self._signature_fun(file_path)
-        return signature == self._get_signature(file_name)
+        return signature == await self._get_signature(file_name)
 
-    def get_granule_status(self,
-                           file_path: str,
-                           date_from: datetime = None,
-                           date_to: datetime = None) -> GranuleStatus:
+    async def get_granule_status(self,
+                                 file_path: str,
+                                 date_from: datetime = None,
+                                 date_to: datetime = None) -> GranuleStatus:
         """
         Get the history status of a granule. DESIRED_FORWARD_PROCESSING means the granule has not yet been ingested
         and and is newer than the newest granule that was ingested (see IngestionHistory.latest_ingested_mtime).
@@ -96,21 +96,21 @@ class IngestionHistory(ABC):
         """
         if self._in_time_range(file_path, date_from=self.latest_ingested_mtime()):
             return GranuleStatus.DESIRED_FORWARD_PROCESSING
-        elif self._in_time_range(file_path, date_from, date_to) and not self.already_ingested(file_path):
+        elif self._in_time_range(file_path, date_from, date_to) and not await self.already_ingested(file_path):
             return GranuleStatus.DESIRED_HISTORICAL
         else:
             return GranuleStatus.UNDESIRED
 
     @abstractmethod
-    def _save_latest_timestamp(self):
+    async def _save_latest_timestamp(self):
         pass
 
     @abstractmethod
-    def _push_record(self, file_name, signature):
+    async def _push_record(self, file_name, signature):
         pass
 
     @abstractmethod
-    def _get_signature(self, file_name):
+    async def _get_signature(self, file_name):
         pass
 
     @staticmethod
diff --git a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
index 79d6eef..59f5cd7 100644
--- a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
@@ -4,10 +4,12 @@ import logging
 import pysolr
 import requests
 
+from common.async_utils.AsyncUtils import run_in_executor
 from collection_manager.services.history_manager.IngestionHistory import IngestionHistory
 from collection_manager.services.history_manager.IngestionHistory import IngestionHistoryBuilder
 from collection_manager.services.history_manager.IngestionHistory import md5sum_from_filepath
 
+logging.getLogger("pysolr").setLevel(logging.WARNING)
 logger = logging.getLogger(__name__)
 
 
@@ -46,6 +48,7 @@ class SolrIngestionHistory(IngestionHistory):
     def __del__(self):
         self._req_session.close()
 
+    @run_in_executor
     def _push_record(self, file_name, signature):
         hash_id = doc_key(self._dataset_id, file_name)
         self._solr_granules.delete(q=f"id:{hash_id}")
@@ -57,6 +60,7 @@ class SolrIngestionHistory(IngestionHistory):
         self._solr_granules.commit()
         return None
 
+    @run_in_executor
     def _save_latest_timestamp(self):
         if self._solr_datasets:
             self._solr_datasets.delete(q=f"id:{self._dataset_id}")
@@ -73,6 +77,7 @@ class SolrIngestionHistory(IngestionHistory):
         else:
             return None
 
+    @run_in_executor
     def _get_signature(self, file_name):
         hash_id = doc_key(self._dataset_id, file_name)
         results = self._solr_granules.search(q=f"id:{hash_id}")
@@ -110,9 +115,6 @@ class SolrIngestionHistory(IngestionHistory):
                 self._add_field(schema_endpoint, "granule_s", "string")
                 self._add_field(schema_endpoint, "granule_signature_s", "string")
 
-            else:
-                logger.info(f"collection {self._granule_collection_name} already exists")
-
             if self._dataset_collection_name not in existing_collections:
                 # Create collection
                 payload = {'action': 'CREATE',
@@ -128,9 +130,6 @@ class SolrIngestionHistory(IngestionHistory):
                 self._add_field(schema_endpoint, "dataset_s", "string")
                 self._add_field(schema_endpoint, "latest_update_l", "TrieLongField")
 
-            else:
-                logger.info(f"collection {self._dataset_collection_name} already exists")
-
         except requests.exceptions.RequestException as e:
             logger.error(f"solr instance unreachable {self._solr_url}")
             raise e
diff --git a/collection_manager/docker/Dockerfile b/collection_manager/docker/Dockerfile
index ce1b577..2a57784 100644
--- a/collection_manager/docker/Dockerfile
+++ b/collection_manager/docker/Dockerfile
@@ -5,12 +5,14 @@ RUN curl -s https://packages.cloud.google.com/apt/doc/apt-key.gpg | apt-key add
 RUN echo "deb https://apt.kubernetes.io/ kubernetes-xenial main" | tee -a /etc/apt/sources.list.d/kubernetes.list
 RUN apt-get update && apt-get install -y kubectl
 
-COPY /collection_manager /collection_manager/collection_manager
-COPY /setup.py /collection_manager/setup.py
-COPY /requirements.txt /collection_manager/requirements.txt
-COPY /README.md /collection_manager/README.md
-COPY /docker/entrypoint.sh /entrypoint.sh
+COPY common /common
+COPY collection_manager/collection_manager /collection_manager/collection_manager
+COPY collection_manager/setup.py /collection_manager/setup.py
+COPY collection_manager/requirements.txt /collection_manager/requirements.txt
+COPY collection_manager/README.md /collection_manager/README.md
+COPY collection_manager/docker/entrypoint.sh /entrypoint.sh
 
+RUN cd /common && python setup.py install
 RUN cd /collection_manager && python setup.py install
 
 ENTRYPOINT ["/bin/bash", "/entrypoint.sh"]
diff --git a/collection_manager/requirements.txt b/collection_manager/requirements.txt
index 7e47c51..ee12c89 100644
--- a/collection_manager/requirements.txt
+++ b/collection_manager/requirements.txt
@@ -1,6 +1,6 @@
 PyYAML==5.3.1
 pystache==0.5.4
-pysolr==3.8.1
+pysolr==3.9.0
 watchdog==0.10.2
 requests==2.23.0
 aio-pika==6.6.1
diff --git a/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py b/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
index d2ad45c..07ab0e1 100644
--- a/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
+++ b/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
@@ -3,53 +3,61 @@ import pathlib
 import tempfile
 import unittest
 
-from collection_manager.services.history_manager import FileIngestionHistory, md5sum_from_filepath
+from collection_manager.services.history_manager import (FileIngestionHistory,
+                                                         md5sum_from_filepath)
+
+from common.async_test_utils.AsyncTestUtils import async_test
 
 DATASET_ID = "zobi_la_mouche"
 
 
 class TestFileIngestionHistory(unittest.TestCase):
 
-    def test_get_md5sum(self):
+    @async_test
+    async def test_get_md5sum(self):
         with tempfile.TemporaryDirectory() as history_dir:
             ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
-            ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
-            result = ingestion_history._get_signature("blue")
+            await ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
+            result = await ingestion_history._get_signature("blue")
             self.assertEqual(result, "12weeukrhbwerqu7wier")
 
-    def test_get_missing_md5sum(self):
+    @async_test
+    async def test_get_missing_md5sum(self):
         with tempfile.TemporaryDirectory() as history_dir:
             ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
-            ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
-            result = ingestion_history._get_signature("green")
+            await ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
+            result = await ingestion_history._get_signature("green")
             self.assertIsNone(result)
 
-    def test_already_ingested(self):
+    @async_test
+    async def test_already_ingested(self):
         with tempfile.TemporaryDirectory() as history_dir:
             ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
             # history_manager with this file
             current_file_path = pathlib.Path(__file__)
-            ingestion_history.push(str(current_file_path))
-            self.assertTrue(ingestion_history.already_ingested(str(current_file_path)))
+            await ingestion_history.push(str(current_file_path))
+            self.assertTrue(await ingestion_history.already_ingested(str(current_file_path)))
 
             del ingestion_history
 
-    def test_already_ingested_with_latest_modifcation_signature(self):
+    @async_test
+    async def test_already_ingested_with_latest_modifcation_signature(self):
         with tempfile.TemporaryDirectory() as history_dir:
             ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, os.path.getmtime)
             # history_manager with this file
             current_file_path = pathlib.Path(__file__)
-            ingestion_history.push(str(current_file_path))
-            self.assertTrue(ingestion_history.already_ingested(str(current_file_path)))
+            await ingestion_history.push(str(current_file_path))
+            self.assertTrue(await ingestion_history.already_ingested(str(current_file_path)))
 
             del ingestion_history
 
-    def test_already_ingested_is_false(self):
+    @async_test
+    async def test_already_ingested_is_false(self):
         with tempfile.TemporaryDirectory() as history_dir:
             ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
             # history_manager with this file
             current_file_path = pathlib.Path(__file__)
-            self.assertFalse(ingestion_history.already_ingested(str(current_file_path)))
+            self.assertFalse(await ingestion_history.already_ingested(str(current_file_path)))
 
 
 if __name__ == '__main__':
diff --git a/collection_manager/tests/services/test_CollectionProcessor.py b/collection_manager/tests/services/test_CollectionProcessor.py
index aa143f3..a7059d6 100644
--- a/collection_manager/tests/services/test_CollectionProcessor.py
+++ b/collection_manager/tests/services/test_CollectionProcessor.py
@@ -65,7 +65,7 @@ class TestCollectionProcessor(unittest.TestCase):
         self.assertEqual(filled, expected)
 
     @async_test
-    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', new_callable=AsyncMock)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
     @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
     async def test_process_granule_with_historical_granule(self, mock_publisher, mock_history_builder, mock_history):
@@ -87,10 +87,12 @@ class TestCollectionProcessor(unittest.TestCase):
         mock_history.push.assert_called()
 
     @async_test
-    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
-    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', new_callable=AsyncMock)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder',  autospec=True)
     @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
-    async def test_process_granule_with_forward_processing_granule(self, mock_publisher, mock_history_builder,
+    async def test_process_granule_with_forward_processing_granule(self,
+                                                                   mock_publisher,
+                                                                   mock_history_builder,
                                                                    mock_history):
         mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_FORWARD_PROCESSING
         mock_history_builder.build.return_value = mock_history
@@ -110,11 +112,11 @@ class TestCollectionProcessor(unittest.TestCase):
         mock_history.push.assert_called()
 
     @async_test
-    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', new_callable=AsyncMock)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
     @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
     async def test_process_granule_with_forward_processing_granule_and_no_priority(self, mock_publisher,
-                                                                             mock_history_builder, mock_history):
+                                                                                   mock_history_builder, mock_history):
         mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_FORWARD_PROCESSING
         mock_history_builder.build.return_value = mock_history
 
@@ -132,7 +134,7 @@ class TestCollectionProcessor(unittest.TestCase):
         mock_history.push.assert_called()
 
     @async_test
-    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', new_callable=AsyncMock)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
     @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
     async def test_process_granule_with_undesired_granule(self, mock_publisher, mock_history_builder, mock_history):
diff --git a/collection_manager/tests/services/test_CollectionWatcher.py b/collection_manager/tests/services/test_CollectionWatcher.py
index 14e7c3c..c9a75c0 100644
--- a/collection_manager/tests/services/test_CollectionWatcher.py
+++ b/collection_manager/tests/services/test_CollectionWatcher.py
@@ -210,4 +210,3 @@ collections:
         callback = AsyncMock()
         await CollectionWatcher._run_periodically(None, 0.1, callback)
         await AsyncAssert.assert_called_within_timeout(callback, timeout_sec=0.3, call_count=2)
-
diff --git a/common/common/async_utils/AsyncUtils.py b/common/common/async_utils/AsyncUtils.py
new file mode 100644
index 0000000..5fefd45
--- /dev/null
+++ b/common/common/async_utils/AsyncUtils.py
@@ -0,0 +1,11 @@
+import asyncio
+import functools
+
+
+def run_in_executor(f):
+    @functools.wraps(f)
+    def inner(*args, **kwargs):
+        loop = asyncio.get_running_loop()
+        return loop.run_in_executor(None, lambda: f(*args, **kwargs))
+
+    return inner
diff --git a/common/common/async_utils/__init__.py b/common/common/async_utils/__init__.py
new file mode 100644
index 0000000..9a468e0
--- /dev/null
+++ b/common/common/async_utils/__init__.py
@@ -0,0 +1 @@
+from .AsyncUtils import run_in_executor
diff --git a/granule_ingester/README.md b/granule_ingester/README.md
index 112f52d..1339835 100644
--- a/granule_ingester/README.md
+++ b/granule_ingester/README.md
@@ -12,23 +12,24 @@ data to Cassandra and Solr.
 Python 3.7
 
 ## Building the service
-From `incubator-sdap-ingester/granule_ingester`, run:
-
-    $ python setup.py install
+From `incubator-sdap-ingester`, run:
+    $ cd common && python setup.py install
+    $ cd ../granule_ingester && python setup.py install
     
 
 ## Launching the service
-From `incubator-sdap-ingester/granule_ingester`, run:
+From `incubator-sdap-ingester`, run:
 
-    $ python granule_ingester/main.py -h
+    $ python granule_ingester/granule_ingester/main.py -h
     
 ## Running the tests
-From `incubator-sdap-ingester/granule_ingester`, run:
+From `incubator-sdap-ingester`, run:
 
-    $ pip install pytest
-    $ pytest
+    $ cd common && python setup.py install
+    $ cd ../granule_ingester && python setup.py install
+    $ pip install pytest && pytest
     
 ## Building the Docker image
-From `incubator-sdap-ingester/granule_ingester`, run:
+From `incubator-sdap-ingester`, run:
 
-    $ docker build . -f docker/Dockerfile -t nexusjpl/granule-ingester
+    $ docker build . -f granule_ingester/docker/Dockerfile -t nexusjpl/granule-ingester
diff --git a/granule_ingester/docker/Dockerfile b/granule_ingester/docker/Dockerfile
index 4b25318..57bacff 100644
--- a/granule_ingester/docker/Dockerfile
+++ b/granule_ingester/docker/Dockerfile
@@ -6,14 +6,16 @@ ENV PATH="/opt/conda/bin:$PATH"
 
 RUN apk update --no-cache && apk add --no-cache --virtual .build-deps git openjdk8
 
-COPY /granule_ingester /sdap/granule_ingester
-COPY /setup.py /sdap/setup.py
-COPY /requirements.txt /sdap/requirements.txt
-COPY /conda-requirements.txt /sdap/conda-requirements.txt
-COPY /docker/install_nexusproto.sh /install_nexusproto.sh
-COPY /docker/entrypoint.sh /entrypoint.sh
+COPY common /common
+COPY granule_ingester/granule_ingester /sdap/granule_ingester
+COPY granule_ingester/setup.py /sdap/setup.py
+COPY granule_ingester/requirements.txt /sdap/requirements.txt
+COPY granule_ingester/conda-requirements.txt /sdap/conda-requirements.txt
+COPY granule_ingester/docker/install_nexusproto.sh /install_nexusproto.sh
+COPY granule_ingester/docker/entrypoint.sh /entrypoint.sh
 
 RUN ./install_nexusproto.sh
+RUN cd /common && python setup.py install
 RUN cd /sdap && python setup.py install
 
 RUN apk del .build-deps
diff --git a/granule_ingester/granule_ingester/writers/SolrStore.py b/granule_ingester/granule_ingester/writers/SolrStore.py
index e098672..67532b5 100644
--- a/granule_ingester/granule_ingester/writers/SolrStore.py
+++ b/granule_ingester/granule_ingester/writers/SolrStore.py
@@ -23,25 +23,18 @@ from pathlib import Path
 from typing import Dict
 
 import pysolr
-from kazoo.handlers.threading import KazooTimeoutError
 from kazoo.exceptions import NoNodeError
-from nexusproto.DataTile_pb2 import TileSummary, NexusTile
+from kazoo.handlers.threading import KazooTimeoutError
 
-from granule_ingester.exceptions import SolrFailedHealthCheckError, SolrLostConnectionError
+from common.async_utils.AsyncUtils import run_in_executor
+from granule_ingester.exceptions import (SolrFailedHealthCheckError,
+                                         SolrLostConnectionError)
 from granule_ingester.writers.MetadataStore import MetadataStore
+from nexusproto.DataTile_pb2 import NexusTile, TileSummary
 
 logger = logging.getLogger(__name__)
 
 
-def run_in_executor(f):
-    @functools.wraps(f)
-    def inner(*args, **kwargs):
-        loop = asyncio.get_running_loop()
-        return loop.run_in_executor(None, lambda: f(*args, **kwargs))
-
-    return inner
-
-
 class SolrStore(MetadataStore):
     def __init__(self, solr_url=None, zk_url=None):
         super().__init__()

[incubator-sdap-ingester] 19/33: SDAP-291: Fix netcdf parsing error by freezing pandas at 1.0.4 (#20)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 97d5a2c9909fcb9dc26e9de90c7dbfa446032baf
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Wed Oct 21 15:20:39 2020 -0700

    SDAP-291: Fix netcdf parsing error by freezing pandas at 1.0.4 (#20)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 granule_ingester/conda-requirements.txt | 1 +
 1 file changed, 1 insertion(+)

diff --git a/granule_ingester/conda-requirements.txt b/granule_ingester/conda-requirements.txt
index fafd6f3..07afb17 100644
--- a/granule_ingester/conda-requirements.txt
+++ b/granule_ingester/conda-requirements.txt
@@ -1,6 +1,7 @@
 numpy==1.15.4
 scipy
 netcdf4==1.5.3
+pandas=1.0.4
 pytz==2019.3
 xarray
 pyyaml==5.3.1

[incubator-sdap-ingester] 30/33: add lfs support for test unit file ()nc, nc4, h5

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit e2390ce69dcad98a63ed14275bf31b6997a07759
Author: thomas loubrieu <th...@jpl.nasa.gov>
AuthorDate: Thu Apr 8 10:45:18 2021 -0700

    add lfs support for test unit file ()nc, nc4, h5
---
 granule_ingester/.gitattributes                    |   3 +++
 ...4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc | Bin 1057327 -> 132 bytes
 granule_ingester/tests/granules/OBP_2017_01.nc     | Bin 2110135 -> 132 bytes
 granule_ingester/tests/granules/OBP_native_grid.nc | Bin 1285094 -> 132 bytes
 .../SMAP_L2B_SSS_04892_20160101T005507_R13080.h5   | Bin 18672352 -> 133 bytes
 granule_ingester/tests/granules/THETA_199201.nc    | Bin 4255957 -> 132 bytes
 granule_ingester/tests/granules/empty_mur.nc4      | Bin 60937 -> 130 bytes
 .../tests/granules/not_empty_ascatb.nc4            | Bin 78036 -> 130 bytes
 .../tests/granules/not_empty_avhrr.nc4             | Bin 49511 -> 130 bytes
 granule_ingester/tests/granules/not_empty_ccmp.nc  | Bin 206870 -> 131 bytes
 granule_ingester/tests/granules/not_empty_mur.nc4  | Bin 60907 -> 130 bytes
 granule_ingester/tests/granules/not_empty_smap.h5  | Bin 3000192 -> 132 bytes
 granule_ingester/tests/granules/not_empty_wswm.nc  | Bin 1041568 -> 132 bytes
 13 files changed, 3 insertions(+)

diff --git a/granule_ingester/.gitattributes b/granule_ingester/.gitattributes
new file mode 100644
index 0000000..5ab5d26
--- /dev/null
+++ b/granule_ingester/.gitattributes
@@ -0,0 +1,3 @@
+*.nc filter=lfs diff=lfs merge=lfs -text
+*.nc4 filter=lfs diff=lfs merge=lfs -text
+*.h5 filter=lfs diff=lfs merge=lfs -text
diff --git a/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc b/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc
index 4935c81..6996466 100644
Binary files a/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc and b/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc differ
diff --git a/granule_ingester/tests/granules/OBP_2017_01.nc b/granule_ingester/tests/granules/OBP_2017_01.nc
index 8c9b584..8401b34 100644
Binary files a/granule_ingester/tests/granules/OBP_2017_01.nc and b/granule_ingester/tests/granules/OBP_2017_01.nc differ
diff --git a/granule_ingester/tests/granules/OBP_native_grid.nc b/granule_ingester/tests/granules/OBP_native_grid.nc
index addb8a0..f4dbe6a 100755
Binary files a/granule_ingester/tests/granules/OBP_native_grid.nc and b/granule_ingester/tests/granules/OBP_native_grid.nc differ
diff --git a/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 b/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5
index 11815dd..8a5d950 100644
Binary files a/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 and b/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 differ
diff --git a/granule_ingester/tests/granules/THETA_199201.nc b/granule_ingester/tests/granules/THETA_199201.nc
index ad92a61..d650b72 100644
Binary files a/granule_ingester/tests/granules/THETA_199201.nc and b/granule_ingester/tests/granules/THETA_199201.nc differ
diff --git a/granule_ingester/tests/granules/empty_mur.nc4 b/granule_ingester/tests/granules/empty_mur.nc4
index f65c808..96c4980 100644
Binary files a/granule_ingester/tests/granules/empty_mur.nc4 and b/granule_ingester/tests/granules/empty_mur.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_ascatb.nc4 b/granule_ingester/tests/granules/not_empty_ascatb.nc4
index d8ef90b..21b48cc 100644
Binary files a/granule_ingester/tests/granules/not_empty_ascatb.nc4 and b/granule_ingester/tests/granules/not_empty_ascatb.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_avhrr.nc4 b/granule_ingester/tests/granules/not_empty_avhrr.nc4
index af24071..f25f916 100644
Binary files a/granule_ingester/tests/granules/not_empty_avhrr.nc4 and b/granule_ingester/tests/granules/not_empty_avhrr.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_ccmp.nc b/granule_ingester/tests/granules/not_empty_ccmp.nc
index b7b491d..5a215d3 100644
Binary files a/granule_ingester/tests/granules/not_empty_ccmp.nc and b/granule_ingester/tests/granules/not_empty_ccmp.nc differ
diff --git a/granule_ingester/tests/granules/not_empty_mur.nc4 b/granule_ingester/tests/granules/not_empty_mur.nc4
index 09d31fd..7133697 100644
Binary files a/granule_ingester/tests/granules/not_empty_mur.nc4 and b/granule_ingester/tests/granules/not_empty_mur.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_smap.h5 b/granule_ingester/tests/granules/not_empty_smap.h5
index 956cbc5..613daa9 100644
Binary files a/granule_ingester/tests/granules/not_empty_smap.h5 and b/granule_ingester/tests/granules/not_empty_smap.h5 differ
diff --git a/granule_ingester/tests/granules/not_empty_wswm.nc b/granule_ingester/tests/granules/not_empty_wswm.nc
index ce0ebcc..5bc5d6f 100644
Binary files a/granule_ingester/tests/granules/not_empty_wswm.nc and b/granule_ingester/tests/granules/not_empty_wswm.nc differ

[incubator-sdap-ingester] 21/33: SDAP-286: Add processor module to Granule Ingester to properly handle longitudes between 180-360deg (#23)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit dba7a01ffc6da88e45590960ff04a20ff1d7cae2
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Nov 3 17:10:28 2020 -0800

    SDAP-286: Add processor module to Granule Ingester to properly handle longitudes between 180-360deg (#23)
---
 .../services/CollectionProcessor.py                |  1 +
 .../granule_ingester/pipeline/Modules.py           |  9 +++--
 .../processors/Subtract180FromLongitude.py         | 38 ++++++++++++++++++++++
 .../granule_ingester/processors/__init__.py        |  1 +
 4 files changed, 47 insertions(+), 2 deletions(-)

diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index f08ade9..975f50c 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -88,6 +88,7 @@ class CollectionProcessor:
                     **dict(collection.dimension_names),
                 },
                 {'name': 'emptyTileFilter'},
+                {'name': 'subtract180FromLongitude'},
                 {
                     'name': 'tileSummary',
                     'dataset_name': collection.dataset_id
diff --git a/granule_ingester/granule_ingester/pipeline/Modules.py b/granule_ingester/granule_ingester/pipeline/Modules.py
index d1950dc..5db706b 100644
--- a/granule_ingester/granule_ingester/pipeline/Modules.py
+++ b/granule_ingester/granule_ingester/pipeline/Modules.py
@@ -1,4 +1,8 @@
-from granule_ingester.processors import GenerateTileId, TileSummarizingProcessor, EmptyTileFilter, KelvinToCelsius
+from granule_ingester.processors import (GenerateTileId,
+                                         TileSummarizingProcessor,
+                                         EmptyTileFilter,
+                                         KelvinToCelsius,
+                                         Subtract180FromLongitude)
 from granule_ingester.processors.reading_processors import (EccoReadingProcessor,
                                                             GridReadingProcessor,
                                                             SwathReadingProcessor,
@@ -16,5 +20,6 @@ modules = {
     "Swath": SwathReadingProcessor,
     "tileSummary": TileSummarizingProcessor,
     "emptyTileFilter": EmptyTileFilter,
-    "kelvinToCelsius": KelvinToCelsius
+    "kelvinToCelsius": KelvinToCelsius,
+    "subtract180FromLongitude": Subtract180FromLongitude
 }
diff --git a/granule_ingester/granule_ingester/processors/Subtract180FromLongitude.py b/granule_ingester/granule_ingester/processors/Subtract180FromLongitude.py
new file mode 100644
index 0000000..8dada92
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/Subtract180FromLongitude.py
@@ -0,0 +1,38 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 granule_ingester.processors.TileProcessor import TileProcessor
+from nexusproto.serialization import from_shaped_array, to_shaped_array
+
+
+class Subtract180FromLongitude(TileProcessor):
+    def process(self, tile, *args, **kwargs):
+        """
+        This method will transform longitude values in degrees_east from 0 TO 360 to -180 to 180
+        :param self:
+        :param nexus_tile: The nexus_tile
+        :return: Tile data with altered longitude values
+        """
+
+        the_tile_type = tile.tile.WhichOneof("tile_type")
+        the_tile_data = getattr(tile.tile, the_tile_type)
+        longitudes = from_shaped_array(the_tile_data.longitude)
+
+        # Only subtract 360 if the longitude is greater than 180
+        longitudes[longitudes > 180] -= 360
+
+        the_tile_data.longitude.CopyFrom(to_shaped_array(longitudes))
+
+        return tile
diff --git a/granule_ingester/granule_ingester/processors/__init__.py b/granule_ingester/granule_ingester/processors/__init__.py
index 592d8ea..ffd73b3 100644
--- a/granule_ingester/granule_ingester/processors/__init__.py
+++ b/granule_ingester/granule_ingester/processors/__init__.py
@@ -3,3 +3,4 @@ from granule_ingester.processors.GenerateTileId import GenerateTileId
 from granule_ingester.processors.TileProcessor import TileProcessor
 from granule_ingester.processors.TileSummarizingProcessor import TileSummarizingProcessor
 from granule_ingester.processors.kelvintocelsius import KelvinToCelsius
+from granule_ingester.processors.Subtract180FromLongitude import Subtract180FromLongitude

[incubator-sdap-ingester] 20/33: SDAP-291: Fix netcdf parsing error by freezing pandas at 1.0.4 (#21)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 0586175860fbf08a23f9b3d680c5f898c81d73f0
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Wed Oct 21 16:49:28 2020 -0700

    SDAP-291: Fix netcdf parsing error by freezing pandas at 1.0.4 (#21)
---
 granule_ingester/conda-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/granule_ingester/conda-requirements.txt b/granule_ingester/conda-requirements.txt
index 07afb17..da92b1e 100644
--- a/granule_ingester/conda-requirements.txt
+++ b/granule_ingester/conda-requirements.txt
@@ -1,7 +1,7 @@
 numpy==1.15.4
 scipy
 netcdf4==1.5.3
-pandas=1.0.4
+pandas==1.0.4
 pytz==2019.3
 xarray
 pyyaml==5.3.1

[incubator-sdap-ingester] 12/33: SDAP-273: Configure max threads in Granule Ingester (#13)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit ab6998b0e71a745caae080bca48230937bf39818
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Aug 4 19:14:22 2020 -0700

    SDAP-273: Configure max threads in Granule Ingester (#13)
---
 granule_ingester/docker/entrypoint.sh              | 19 ++++++++-------
 .../granule_ingester/consumer/Consumer.py          | 15 ++++++++----
 granule_ingester/granule_ingester/main.py          | 24 +++++++++++--------
 .../granule_ingester/pipeline/Pipeline.py          | 28 ++++++++++++++--------
 4 files changed, 52 insertions(+), 34 deletions(-)

diff --git a/granule_ingester/docker/entrypoint.sh b/granule_ingester/docker/entrypoint.sh
index b703ee3..04ed15c 100644
--- a/granule_ingester/docker/entrypoint.sh
+++ b/granule_ingester/docker/entrypoint.sh
@@ -1,12 +1,13 @@
 #!/bin/sh
 
 python /sdap/granule_ingester/main.py \
-  $([[ ! -z "$RABBITMQ_HOST" ]] && echo --rabbitmq_host=$RABBITMQ_HOST) \
-  $([[ ! -z "$RABBITMQ_USERNAME" ]] && echo --rabbitmq_username=$RABBITMQ_USERNAME) \
-  $([[ ! -z "$RABBITMQ_PASSWORD" ]] && echo --rabbitmq_password=$RABBITMQ_PASSWORD) \
-  $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq_queue=$RABBITMQ_QUEUE) \
-  $([[ ! -z "$CASSANDRA_CONTACT_POINTS" ]] && echo --cassandra_contact_points=$CASSANDRA_CONTACT_POINTS) \
-  $([[ ! -z "$CASSANDRA_PORT" ]] && echo --cassandra_port=$CASSANDRA_PORT) \
-  $([[ ! -z "$CASSANDRA_USERNAME" ]] && echo --cassandra_username=$CASSANDRA_USERNAME) \
-  $([[ ! -z "$CASSANDRA_PASSWORD" ]] && echo --cassandra_password=$CASSANDRA_PASSWORD) \
-  $([[ ! -z "$SOLR_HOST_AND_PORT" ]] && echo --solr_host_and_port=$SOLR_HOST_AND_PORT)
+  $([[ ! -z "$RABBITMQ_HOST" ]] && echo --rabbitmq-host=$RABBITMQ_HOST) \
+  $([[ ! -z "$RABBITMQ_USERNAME" ]] && echo --rabbitmq-username=$RABBITMQ_USERNAME) \
+  $([[ ! -z "$RABBITMQ_PASSWORD" ]] && echo --rabbitmq-password=$RABBITMQ_PASSWORD) \
+  $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq-queue=$RABBITMQ_QUEUE) \
+  $([[ ! -z "$CASSANDRA_CONTACT_POINTS" ]] && echo --cassandra-contact-points=$CASSANDRA_CONTACT_POINTS) \
+  $([[ ! -z "$CASSANDRA_PORT" ]] && echo --cassandra-port=$CASSANDRA_PORT) \
+  $([[ ! -z "$CASSANDRA_USERNAME" ]] && echo --cassandra-username=$CASSANDRA_USERNAME) \
+  $([[ ! -z "$CASSANDRA_PASSWORD" ]] && echo --cassandra-password=$CASSANDRA_PASSWORD) \
+  $([[ ! -z "$SOLR_HOST_AND_PORT" ]] && echo --solr-host-and-port=$SOLR_HOST_AND_PORT) \
+  $([[ ! -z "$MAX_THREADS" ]] && echo --max-threads=$MAX_THREADS)
diff --git a/granule_ingester/granule_ingester/consumer/Consumer.py b/granule_ingester/granule_ingester/consumer/Consumer.py
index 75d347a..5df51fe 100644
--- a/granule_ingester/granule_ingester/consumer/Consumer.py
+++ b/granule_ingester/granule_ingester/consumer/Consumer.py
@@ -46,7 +46,7 @@ class Consumer(HealthCheck):
             connection = await self._get_connection()
             await connection.close()
             return True
-        except:
+        except Exception:
             logger.error("Cannot connect to RabbitMQ! Connection string was {}".format(self._connection_string))
             return False
 
@@ -64,25 +64,30 @@ class Consumer(HealthCheck):
     @staticmethod
     async def _received_message(message: aio_pika.IncomingMessage,
                                 data_store_factory,
-                                metadata_store_factory):
+                                metadata_store_factory,
+                                pipeline_max_concurrency: int):
         logger.info("Received a job from the queue. Starting pipeline.")
         try:
             config_str = message.body.decode("utf-8")
             logger.debug(config_str)
             pipeline = Pipeline.from_string(config_str=config_str,
                                             data_store_factory=data_store_factory,
-                                            metadata_store_factory=metadata_store_factory)
+                                            metadata_store_factory=metadata_store_factory,
+                                            max_concurrency=pipeline_max_concurrency)
             await pipeline.run()
             message.ack()
         except Exception as e:
             message.reject(requeue=True)
             logger.error("Processing message failed. Message will be re-queued. The exception was:\n{}".format(e))
 
-    async def start_consuming(self):
+    async def start_consuming(self, pipeline_max_concurrency=16):
         channel = await self._connection.channel()
         await channel.set_qos(prefetch_count=1)
         queue = await channel.declare_queue(self._rabbitmq_queue, durable=True)
 
         async with queue.iterator() as queue_iter:
             async for message in queue_iter:
-                await self._received_message(message, self._data_store_factory, self._metadata_store_factory)
+                await self._received_message(message,
+                                             self._data_store_factory,
+                                             self._metadata_store_factory,
+                                             pipeline_max_concurrency)
diff --git a/granule_ingester/granule_ingester/main.py b/granule_ingester/granule_ingester/main.py
index 9010e33..b54cffd 100644
--- a/granule_ingester/granule_ingester/main.py
+++ b/granule_ingester/granule_ingester/main.py
@@ -47,43 +47,47 @@ async def run_health_checks(dependencies: List[HealthCheck]):
 async def main():
     parser = argparse.ArgumentParser(description='Listen to RabbitMQ for granule ingestion instructions, and process '
                                                  'and ingest a granule for each message that comes through.')
-    parser.add_argument('--rabbitmq_host',
+    parser.add_argument('--rabbitmq-host',
                         default='localhost',
                         metavar='HOST',
                         help='RabbitMQ hostname to connect to. (Default: "localhost")')
-    parser.add_argument('--rabbitmq_username',
+    parser.add_argument('--rabbitmq-username',
                         default='guest',
                         metavar='USERNAME',
                         help='RabbitMQ username. (Default: "guest")')
-    parser.add_argument('--rabbitmq_password',
+    parser.add_argument('--rabbitmq-password',
                         default='guest',
                         metavar='PASSWORD',
                         help='RabbitMQ password. (Default: "guest")')
-    parser.add_argument('--rabbitmq_queue',
+    parser.add_argument('--rabbitmq-queue',
                         default="nexus",
                         metavar="QUEUE",
                         help='Name of the RabbitMQ queue to consume from. (Default: "nexus")')
-    parser.add_argument('--cassandra_contact_points',
+    parser.add_argument('--cassandra-contact-points',
                         default=['localhost'],
                         metavar="HOST",
                         nargs='+',
                         help='List of one or more Cassandra contact points, separated by spaces. (Default: "localhost")')
-    parser.add_argument('--cassandra_port',
+    parser.add_argument('--cassandra-port',
                         default=9042,
                         metavar="PORT",
                         help='Cassandra port. (Default: 9042)')
-    parser.add_argument('--cassandra_username',
+    parser.add_argument('--cassandra-username',
                         metavar="USERNAME",
                         default=None,
                         help='Cassandra username. Optional.')
-    parser.add_argument('--cassandra_password',
+    parser.add_argument('--cassandra-password',
                         metavar="PASSWORD",
                         default=None,
                         help='Cassandra password. Optional.')
-    parser.add_argument('--solr_host_and_port',
+    parser.add_argument('--solr-host-and-port',
                         default='http://localhost:8983',
                         metavar='HOST:PORT',
                         help='Solr host and port. (Default: http://localhost:8983)')
+    parser.add_argument('--max-threads',
+                        default=16,
+                        metavar='MAX_THREADS',
+                        help='Maximum number of threads to use when processing granules. (Default: 16)')
     parser.add_argument('-v',
                         '--verbose',
                         action='store_true',
@@ -126,7 +130,7 @@ async def main():
                                 consumer]):
         async with consumer:
             logger.info("All external dependencies have passed the health checks. Now listening to message queue.")
-            await consumer.start_consuming()
+            await consumer.start_consuming(args.max_threads)
     else:
         logger.error("Quitting because not all dependencies passed the health checks.")
 
diff --git a/granule_ingester/granule_ingester/pipeline/Pipeline.py b/granule_ingester/granule_ingester/pipeline/Pipeline.py
index 8f2dd6f..e1e53bf 100644
--- a/granule_ingester/granule_ingester/pipeline/Pipeline.py
+++ b/granule_ingester/granule_ingester/pipeline/Pipeline.py
@@ -15,19 +15,20 @@
 
 
 import logging
+import os
 import time
 from typing import List
 
-import aiomultiprocess
 import xarray as xr
 import yaml
-from nexusproto import DataTile_pb2 as nexusproto
 
+import aiomultiprocess
 from granule_ingester.granule_loaders import GranuleLoader
 from granule_ingester.pipeline.Modules import modules as processor_module_mappings
 from granule_ingester.processors.TileProcessor import TileProcessor
 from granule_ingester.slicers import TileSlicer
 from granule_ingester.writers import DataStore, MetadataStore
+from nexusproto import DataTile_pb2 as nexusproto
 
 logger = logging.getLogger(__name__)
 
@@ -81,36 +82,41 @@ class Pipeline:
                  slicer: TileSlicer,
                  data_store_factory,
                  metadata_store_factory,
-                 tile_processors: List[TileProcessor]):
+                 tile_processors: List[TileProcessor],
+                 max_concurrency: int):
         self._granule_loader = granule_loader
         self._tile_processors = tile_processors
         self._slicer = slicer
         self._data_store_factory = data_store_factory
         self._metadata_store_factory = metadata_store_factory
+        self._max_concurrency = max_concurrency
 
     @classmethod
-    def from_string(cls, config_str: str, data_store_factory, metadata_store_factory):
+    def from_string(cls, config_str: str, data_store_factory, metadata_store_factory, max_concurrency: int = 16):
         config = yaml.load(config_str, yaml.FullLoader)
         return cls._build_pipeline(config,
                                    data_store_factory,
                                    metadata_store_factory,
-                                   processor_module_mappings)
+                                   processor_module_mappings,
+                                   max_concurrency)
 
     @classmethod
-    def from_file(cls, config_path: str, data_store_factory, metadata_store_factory):
+    def from_file(cls, config_path: str, data_store_factory, metadata_store_factory, max_concurrency: int = 16):
         with open(config_path) as config_file:
             config = yaml.load(config_file, yaml.FullLoader)
             return cls._build_pipeline(config,
                                        data_store_factory,
                                        metadata_store_factory,
-                                       processor_module_mappings)
+                                       processor_module_mappings,
+                                       max_concurrency)
 
     @classmethod
     def _build_pipeline(cls,
                         config: dict,
                         data_store_factory,
                         metadata_store_factory,
-                        module_mappings: dict):
+                        module_mappings: dict,
+                        max_concurrency: int):
         granule_loader = GranuleLoader(**config['granule'])
 
         slicer_config = config['slicer']
@@ -121,7 +127,7 @@ class Pipeline:
             module = cls._parse_module(processor_config, module_mappings)
             tile_processors.append(module)
 
-        return cls(granule_loader, slicer, data_store_factory, metadata_store_factory, tile_processors)
+        return cls(granule_loader, slicer, data_store_factory, metadata_store_factory, tile_processors, max_concurrency)
 
     @classmethod
     def _parse_module(cls, module_config: dict, module_mappings: dict):
@@ -142,7 +148,9 @@ class Pipeline:
                                             initargs=(self._tile_processors,
                                                       dataset,
                                                       self._data_store_factory,
-                                                      self._metadata_store_factory)) as pool:
+                                                      self._metadata_store_factory),
+                                            maxtasksperchild=self._max_concurrency,
+                                            childconcurrency=self._max_concurrency) as pool:
                 serialized_tiles = [nexusproto.NexusTile.SerializeToString(tile) for tile in
                                     self._slicer.generate_tiles(dataset, granule_name)]
                 # aiomultiprocess is built on top of the stdlib multiprocessing library, which has the limitation that

[incubator-sdap-ingester] 14/33: SDAP-272: Support connecting to Solr through Zk in Granule Ingester (#12)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit f9462ed6d0e92cf71f3b8cdc13287a627e0da062
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Wed Aug 5 13:08:21 2020 -0700

    SDAP-272: Support connecting to Solr through Zk in Granule Ingester (#12)
---
 granule_ingester/docker/entrypoint.sh              |  1 +
 granule_ingester/granule_ingester/main.py          | 43 ++++++++----
 .../granule_ingester/writers/SolrStore.py          | 81 ++++++++++++++--------
 granule_ingester/requirements.txt                  |  5 +-
 4 files changed, 85 insertions(+), 45 deletions(-)

diff --git a/granule_ingester/docker/entrypoint.sh b/granule_ingester/docker/entrypoint.sh
index 04ed15c..662bd3d 100644
--- a/granule_ingester/docker/entrypoint.sh
+++ b/granule_ingester/docker/entrypoint.sh
@@ -10,4 +10,5 @@ python /sdap/granule_ingester/main.py \
   $([[ ! -z "$CASSANDRA_USERNAME" ]] && echo --cassandra-username=$CASSANDRA_USERNAME) \
   $([[ ! -z "$CASSANDRA_PASSWORD" ]] && echo --cassandra-password=$CASSANDRA_PASSWORD) \
   $([[ ! -z "$SOLR_HOST_AND_PORT" ]] && echo --solr-host-and-port=$SOLR_HOST_AND_PORT) \
+  $([[ ! -z "$ZK_HOST_AND_PORT" ]] && echo --zk_host_and_port=$ZK_HOST_AND_PORT) \
   $([[ ! -z "$MAX_THREADS" ]] && echo --max-threads=$MAX_THREADS)
diff --git a/granule_ingester/granule_ingester/main.py b/granule_ingester/granule_ingester/main.py
index b54cffd..15390fd 100644
--- a/granule_ingester/granule_ingester/main.py
+++ b/granule_ingester/granule_ingester/main.py
@@ -16,13 +16,14 @@
 import argparse
 import asyncio
 import logging
+import sys
 from functools import partial
 from typing import List
 
 from granule_ingester.consumer import Consumer
+from granule_ingester.exceptions import FailedHealthCheckError, LostConnectionError
 from granule_ingester.healthcheck import HealthCheck
-from granule_ingester.writers import CassandraStore
-from granule_ingester.writers import SolrStore
+from granule_ingester.writers import CassandraStore, SolrStore
 
 
 def cassandra_factory(contact_points, port, username, password):
@@ -31,8 +32,8 @@ def cassandra_factory(contact_points, port, username, password):
     return store
 
 
-def solr_factory(solr_host_and_port):
-    store = SolrStore(solr_host_and_port)
+def solr_factory(solr_host_and_port, zk_host_and_port):
+    store = SolrStore(zk_url=zk_host_and_port) if zk_host_and_port else SolrStore(solr_url=solr_host_and_port)
     store.connect()
     return store
 
@@ -44,7 +45,7 @@ async def run_health_checks(dependencies: List[HealthCheck]):
     return True
 
 
-async def main():
+async def main(loop):
     parser = argparse.ArgumentParser(description='Listen to RabbitMQ for granule ingestion instructions, and process '
                                                  'and ingest a granule for each message that comes through.')
     parser.add_argument('--rabbitmq-host',
@@ -84,6 +85,8 @@ async def main():
                         default='http://localhost:8983',
                         metavar='HOST:PORT',
                         help='Solr host and port. (Default: http://localhost:8983)')
+    parser.add_argument('--zk_host_and_port',
+                        metavar="HOST:PORT")
     parser.add_argument('--max-threads',
                         default=16,
                         metavar='MAX_THREADS',
@@ -111,6 +114,7 @@ async def main():
     cassandra_contact_points = args.cassandra_contact_points
     cassandra_port = args.cassandra_port
     solr_host_and_port = args.solr_host_and_port
+    zk_host_and_port = args.zk_host_and_port
 
     consumer = Consumer(rabbitmq_host=args.rabbitmq_host,
                         rabbitmq_username=args.rabbitmq_username,
@@ -121,19 +125,28 @@ async def main():
                                                    cassandra_port,
                                                    cassandra_username,
                                                    cassandra_password),
-                        metadata_store_factory=partial(solr_factory, solr_host_and_port))
-    if await run_health_checks([CassandraStore(cassandra_contact_points,
-                                               cassandra_port,
-                                               cassandra_username,
-                                               cassandra_password),
-                                SolrStore(solr_host_and_port),
-                                consumer]):
+                        metadata_store_factory=partial(solr_factory, solr_host_and_port, zk_host_and_port))
+    try:
+        solr_store = SolrStore(zk_url=zk_host_and_port) if zk_host_and_port else SolrStore(solr_url=solr_host_and_port)
+        await run_health_checks([CassandraStore(cassandra_contact_points,
+                                                cassandra_port,
+                                                cassandra_username,
+                                                cassandra_password),
+                                 solr_store,
+                                 consumer])
         async with consumer:
             logger.info("All external dependencies have passed the health checks. Now listening to message queue.")
             await consumer.start_consuming(args.max_threads)
-    else:
-        logger.error("Quitting because not all dependencies passed the health checks.")
+    except FailedHealthCheckError as e:
+        logger.error(f"Quitting because not all dependencies passed the health checks: {e}")
+    except LostConnectionError as e:
+        logger.error(f"{e} Any messages that were being processed have been re-queued. Quitting.")
+    except Exception as e:
+        logger.exception(f"Shutting down because of an unrecoverable error:\n{e}")
+    finally:
+        sys.exit(1)
 
 
 if __name__ == '__main__':
-    asyncio.run(main())
+    loop = asyncio.get_event_loop()
+    loop.run_until_complete(main(loop))
diff --git a/granule_ingester/granule_ingester/writers/SolrStore.py b/granule_ingester/granule_ingester/writers/SolrStore.py
index 9d6a7f0..e098672 100644
--- a/granule_ingester/granule_ingester/writers/SolrStore.py
+++ b/granule_ingester/granule_ingester/writers/SolrStore.py
@@ -13,64 +13,87 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-
-from asyncio import AbstractEventLoop
-
+import asyncio
+import functools
+import json
 import logging
+from asyncio import AbstractEventLoop
 from datetime import datetime
 from pathlib import Path
 from typing import Dict
 
-import aiohttp
-from nexusproto.DataTile_pb2 import *
-from tenacity import *
+import pysolr
+from kazoo.handlers.threading import KazooTimeoutError
+from kazoo.exceptions import NoNodeError
+from nexusproto.DataTile_pb2 import TileSummary, NexusTile
 
+from granule_ingester.exceptions import SolrFailedHealthCheckError, SolrLostConnectionError
 from granule_ingester.writers.MetadataStore import MetadataStore
 
 logger = logging.getLogger(__name__)
 
 
+def run_in_executor(f):
+    @functools.wraps(f)
+    def inner(*args, **kwargs):
+        loop = asyncio.get_running_loop()
+        return loop.run_in_executor(None, lambda: f(*args, **kwargs))
+
+    return inner
+
+
 class SolrStore(MetadataStore):
-    def __init__(self, host_and_port='http://localhost:8983'):
+    def __init__(self, solr_url=None, zk_url=None):
         super().__init__()
 
         self.TABLE_NAME = "sea_surface_temp"
         self.iso: str = '%Y-%m-%dT%H:%M:%SZ'
 
-        self._host_and_port = host_and_port
+        self._solr_url = solr_url
+        self._zk_url = zk_url
         self.geo_precision: int = 3
-        self.collection: str = "nexustiles"
+        self._collection: str = "nexustiles"
         self.log: logging.Logger = logging.getLogger(__name__)
         self.log.setLevel(logging.DEBUG)
-        self._session = None
+        self._solr = None
+
+    def _get_connection(self) -> pysolr.Solr:
+        if self._zk_url:
+            zk = pysolr.ZooKeeper(f"{self._zk_url}")
+            collections = {}
+            for c in zk.zk.get_children("collections"):
+                collections.update(json.loads(zk.zk.get("collections/{}/state.json".format(c))[0].decode("ascii")))
+            zk.collections = collections
+            return pysolr.SolrCloud(zk, self._collection, always_commit=True)
+        elif self._solr_url:
+            return pysolr.Solr(f'{self._solr_url}/solr/{self._collection}', always_commit=True)
+        else:
+            raise RuntimeError("You must provide either solr_host or zookeeper_host.")
 
     def connect(self, loop: AbstractEventLoop = None):
-        self._session = aiohttp.ClientSession(loop=loop)
+        self._solr = self._get_connection()
 
     async def health_check(self):
         try:
-            async with aiohttp.ClientSession() as session:
-                response = await session.get('{}/solr/{}/admin/ping'.format(self._host_and_port, self.collection))
-                if response.status == 200:
-                    return True
-                else:
-                    logger.error("Solr health check returned status {}.".format(response.status))
-        except aiohttp.ClientConnectionError as e:
-            logger.error("Cannot connect to Solr!")
-
-        return False
+            connection = self._get_connection()
+            connection.ping()
+        except pysolr.SolrError:
+            raise SolrFailedHealthCheckError("Cannot connect to Solr!")
+        except NoNodeError:
+            raise SolrFailedHealthCheckError("Connected to Zookeeper but cannot connect to Solr!")
+        except KazooTimeoutError:
+            raise SolrFailedHealthCheckError("Cannot connect to Zookeeper!")
 
     async def save_metadata(self, nexus_tile: NexusTile) -> None:
         solr_doc = self._build_solr_doc(nexus_tile)
+        await self._save_document(solr_doc)
 
-        await self._save_document(self.collection, solr_doc)
-
-    @retry(stop=stop_after_attempt(5))
-    async def _save_document(self, collection: str, doc: dict):
-        url = '{}/solr/{}/update/json/docs?commit=true'.format(self._host_and_port, collection)
-        response = await self._session.post(url, json=doc)
-        if response.status < 200 or response.status >= 400:
-            raise RuntimeError("Saving data to Solr failed with HTTP status code {}".format(response.status))
+    @run_in_executor
+    def _save_document(self, doc: dict):
+        try:
+            self._solr.add([doc])
+        except pysolr.SolrError:
+            raise SolrLostConnectionError("Lost connection to Solr, and cannot save tiles.")
 
     def _build_solr_doc(self, tile: NexusTile) -> Dict:
         summary: TileSummary = tile.summary
diff --git a/granule_ingester/requirements.txt b/granule_ingester/requirements.txt
index 4d9d4cb..9b06860 100644
--- a/granule_ingester/requirements.txt
+++ b/granule_ingester/requirements.txt
@@ -1,3 +1,6 @@
 cassandra-driver==3.23.0
-aiomultiprocess
+aiomultiprocess==0.7.0
 aioboto3
+tblib==1.6.0
+pysolr==3.9.0
+kazoo==2.8.0
\ No newline at end of file

[incubator-sdap-ingester] 01/33: SDAP-234: Add more test coverage. (#1)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 36b87d0be6d6184fac63ab3556445ebe3e6295c2
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Thu Jun 18 15:53:34 2020 -0700

    SDAP-234: Add more test coverage. (#1)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 .../collection_manager/entities/Collection.py      |  40 ++--
 .../collection_manager/entities/__init__.py        |   3 +-
 .../entities/exceptions/Exceptions.py              |  30 +++
 .../entities/exceptions/__init__.py                |   6 +
 collection_manager/collection_manager/main.py      |  58 +++---
 .../services/CollectionProcessor.py                |   7 +-
 .../services/CollectionWatcher.py                  | 103 +++++++----
 .../{history_manager => entities}/__init__.py      |   0
 .../tests/entities/test_Collection.py              | 139 ++++++++++++++
 .../test_datasetingestionhistoryfile.py            |  64 -------
 collection_manager/tests/resources/collections.yml |  17 ++
 .../tests/resources/collections_alternate.yml      |  17 ++
 .../tests/resources/collections_bad.yml            |  17 ++
 .../tests/resources/data/collections.yml           |  11 --
 .../resources/data/dataset_config_file_ok.yml      |  44 -----
 .../{ => services}/history_manager/__init__.py     |   0
 .../history_manager/test_FileIngestionHistory.py   |  56 ++++++
 .../history_manager/test_SolrIngestionHistory.py}  |   3 +-
 .../tests/services/test_CollectionProcessor.py     | 122 ++++++++++++-
 .../tests/services/test_CollectionWatcher.py       | 203 +++++++++++++++++++++
 20 files changed, 734 insertions(+), 206 deletions(-)

diff --git a/collection_manager/collection_manager/entities/Collection.py b/collection_manager/collection_manager/entities/Collection.py
index d033c69..3976b6d 100644
--- a/collection_manager/collection_manager/entities/Collection.py
+++ b/collection_manager/collection_manager/entities/Collection.py
@@ -1,33 +1,39 @@
 import os
+from dataclasses import dataclass
 from datetime import datetime
 from fnmatch import fnmatch
 from glob import glob
-from typing import NamedTuple, List
+from typing import List, Optional
 
+from collection_manager.entities.exceptions import MissingValueCollectionError
 
-class Collection(NamedTuple):
+
+@dataclass(frozen=True)
+class Collection:
     dataset_id: str
     variable: str
     path: str
     historical_priority: int
-    forward_processing_priority: int
-    date_from: datetime
-    date_to: datetime
+    forward_processing_priority: Optional[int] = None
+    date_from: Optional[datetime] = None
+    date_to: Optional[datetime] = None
 
     @staticmethod
     def from_dict(properties: dict):
-        date_to = datetime.fromisoformat(properties['to']) if 'to' in properties else None
-        date_from = datetime.fromisoformat(properties['from']) if 'from' in properties else None
-
-        collection = Collection(dataset_id=properties['id'],
-                                variable=properties['variable'],
-                                path=properties['path'],
-                                historical_priority=properties['priority'],
-                                forward_processing_priority=properties.get('forward_processing_priority',
-                                                                           properties['priority']),
-                                date_to=date_to,
-                                date_from=date_from)
-        return collection
+        try:
+            date_to = datetime.fromisoformat(properties['to']) if 'to' in properties else None
+            date_from = datetime.fromisoformat(properties['from']) if 'from' in properties else None
+
+            collection = Collection(dataset_id=properties['id'],
+                                    variable=properties['variable'],
+                                    path=properties['path'],
+                                    historical_priority=properties['priority'],
+                                    forward_processing_priority=properties.get('forward-processing-priority', None),
+                                    date_to=date_to,
+                                    date_from=date_from)
+            return collection
+        except KeyError as e:
+            raise MissingValueCollectionError(missing_value=e.args[0])
 
     def directory(self):
         if os.path.isdir(self.path):
diff --git a/collection_manager/collection_manager/entities/__init__.py b/collection_manager/collection_manager/entities/__init__.py
index 9f30603..165341b 100644
--- a/collection_manager/collection_manager/entities/__init__.py
+++ b/collection_manager/collection_manager/entities/__init__.py
@@ -1,2 +1 @@
-
-from .Collection import Collection
\ No newline at end of file
+from .Collection import Collection
diff --git a/collection_manager/collection_manager/entities/exceptions/Exceptions.py b/collection_manager/collection_manager/entities/exceptions/Exceptions.py
new file mode 100644
index 0000000..8e63d24
--- /dev/null
+++ b/collection_manager/collection_manager/entities/exceptions/Exceptions.py
@@ -0,0 +1,30 @@
+class RelativePathError(Exception):
+    pass
+
+
+class YamlParsingError(Exception):
+    pass
+
+
+class CollectionConfigFileNotFoundError(Exception):
+    pass
+
+
+class CollectionError(Exception):
+    def __init__(self, collection=None, message=None):
+        super().__init__(message)
+        self.collection = collection
+
+
+class MissingValueCollectionError(CollectionError):
+    def __init__(self, missing_value, collection=None, message=None):
+        super().__init__(collection, message)
+        self.missing_value = missing_value
+
+
+class ConflictingPathCollectionError(CollectionError):
+    pass
+
+
+class RelativePathCollectionError(CollectionError):
+    pass
diff --git a/collection_manager/collection_manager/entities/exceptions/__init__.py b/collection_manager/collection_manager/entities/exceptions/__init__.py
new file mode 100644
index 0000000..9a22c16
--- /dev/null
+++ b/collection_manager/collection_manager/entities/exceptions/__init__.py
@@ -0,0 +1,6 @@
+from .Exceptions import CollectionConfigFileNotFoundError
+from .Exceptions import ConflictingPathCollectionError
+from .Exceptions import MissingValueCollectionError
+from .Exceptions import RelativePathCollectionError
+from .Exceptions import RelativePathError
+from .Exceptions import YamlParsingError
diff --git a/collection_manager/collection_manager/main.py b/collection_manager/collection_manager/main.py
index fa4d7ce..a10446f 100644
--- a/collection_manager/collection_manager/main.py
+++ b/collection_manager/collection_manager/main.py
@@ -1,5 +1,6 @@
 import argparse
 import logging
+import os
 import time
 
 from collection_manager.services import CollectionProcessor, CollectionWatcher, MessagePublisher
@@ -7,7 +8,13 @@ from collection_manager.services.history_manager import SolrIngestionHistoryBuil
 
 logging.basicConfig(level=logging.INFO)
 logging.getLogger("pika").setLevel(logging.WARNING)
-logger = logging.getLogger(__name__)
+logger = logging.getLogger("collection_manager")
+
+
+def check_path(path) -> str:
+    if not os.path.isabs(path):
+        raise argparse.ArgumentError("Paths must be absolute.")
+    return path
 
 
 def get_args() -> argparse.Namespace:
@@ -16,7 +23,7 @@ def get_args() -> argparse.Namespace:
                         help="refresh interval in seconds to check for new or updated granules",
                         default=300)
     parser.add_argument("--collections",
-                        help="path to collections configuration file",
+                        help="Absolute path to collections configuration file",
                         required=True)
     parser.add_argument('--rabbitmq_host',
                         default='localhost',
@@ -36,32 +43,39 @@ def get_args() -> argparse.Namespace:
                         help='Name of the RabbitMQ queue to consume from. (Default: "nexus")')
     history_group = parser.add_mutually_exclusive_group(required=True)
     history_group.add_argument("--history-path",
-                               help="path to ingestion history local directory")
+                               help="Absolute path to ingestion history local directory")
     history_group.add_argument("--history-url",
-                               help="url to ingestion history solr database")
+                               help="URL to ingestion history solr database")
+
     return parser.parse_args()
 
 
 def main():
-    options = get_args()
-    if options.history_path:
-        history_manager_builder = FileIngestionHistoryBuilder(history_path=options.history_path)
-    else:
-        history_manager_builder = SolrIngestionHistoryBuilder(solr_url=options.history_url)
-    publisher = MessagePublisher(host=options.rabbitmq_host,
-                                 username=options.rabbitmq_username,
-                                 password=options.rabbitmq_password,
-                                 queue=options.rabbitmq_queue)
-    publisher.connect()
-    collection_processor = CollectionProcessor(message_publisher=publisher,
-                                               history_manager_builder=history_manager_builder)
-    collection_watcher = CollectionWatcher(collections_path=options.collections,
-                                           collection_updated_callback=collection_processor.process_collection,
-                                           granule_updated_callback=collection_processor.process_granule)
+    try:
+        options = get_args()
+
+        if options.history_path:
+            history_manager_builder = FileIngestionHistoryBuilder(history_path=options.history_path)
+        else:
+            history_manager_builder = SolrIngestionHistoryBuilder(solr_url=options.history_url)
+        publisher = MessagePublisher(host=options.rabbitmq_host,
+                                     username=options.rabbitmq_username,
+                                     password=options.rabbitmq_password,
+                                     queue=options.rabbitmq_queue)
+        publisher.connect()
+        collection_processor = CollectionProcessor(message_publisher=publisher,
+                                                   history_manager_builder=history_manager_builder)
+        collection_watcher = CollectionWatcher(collections_path=options.collections,
+                                               collection_updated_callback=collection_processor.process_collection,
+                                               granule_updated_callback=collection_processor.process_granule)
+
+        collection_watcher.start_watching()
+        while True:
+            time.sleep(1)
 
-    collection_watcher.start_watching()
-    while True:
-        time.sleep(1)
+    except Exception as e:
+        logger.error(e)
+        return
 
 
 if __name__ == "__main__":
diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index 75a86e2..a81390b 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -50,7 +50,10 @@ class CollectionProcessor:
         if granule_status is GranuleStatus.DESIRED_FORWARD_PROCESSING:
             logger.info(f"New granule '{granule}' detected for forward-processing ingestion "
                         f"in collection '{collection.dataset_id}'.")
-            use_priority = collection.forward_processing_priority
+            if collection.forward_processing_priority is not None:
+                use_priority = collection.forward_processing_priority
+            else:
+                use_priority = collection.historical_priority
         elif granule_status is GranuleStatus.DESIRED_HISTORICAL:
             logger.info(f"New granule '{granule}' detected for historical ingestion in collection "
                         f"'{collection.dataset_id}'.")
@@ -61,7 +64,7 @@ class CollectionProcessor:
             return
 
         dataset_config = self._fill_template(collection, config_template=self._config_template)
-        self._publisher.publish_message(dataset_config, use_priority)
+        self._publisher.publish_message(body=dataset_config, priority=use_priority)
         history_manager.push(granule)
 
     @staticmethod
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index b1fca64..6bbe7d9 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -1,7 +1,7 @@
 import logging
 import os
 from collections import defaultdict
-from typing import List, Dict, Callable, Set
+from typing import Dict, Callable, Set
 
 import yaml
 from watchdog.events import FileSystemEventHandler
@@ -9,6 +9,9 @@ from watchdog.observers import Observer
 from yaml.scanner import ScannerError
 
 from collection_manager.entities import Collection
+from collection_manager.entities.exceptions import RelativePathError, YamlParsingError, \
+    CollectionConfigFileNotFoundError, MissingValueCollectionError, ConflictingPathCollectionError, \
+    RelativePathCollectionError
 
 logger = logging.getLogger(__name__)
 logger.setLevel(logging.DEBUG)
@@ -19,71 +22,96 @@ class CollectionWatcher:
                  collections_path: str,
                  collection_updated_callback: Callable[[Collection], any],
                  granule_updated_callback: Callable[[str, Collection], any]):
+        if not os.path.isabs(collections_path):
+            raise RelativePathError("Collections config  path must be an absolute path.")
+
         self._collections_path = collections_path
         self._collection_updated_callback = collection_updated_callback
         self._granule_updated_callback = granule_updated_callback
-        
+
         self._collections_by_dir: Dict[str, Set[Collection]] = defaultdict(set)
         self._observer = Observer()
 
+        self._granule_watches = set()
+
     def start_watching(self):
         """
         Start observing filesystem events for added/modified granules or changes to the Collections configuration file.
         When an event occurs, call the appropriate callback that was passed in during instantiation.
         :return: None
         """
-        self._observer.schedule(_CollectionEventHandler(file_path=self._collections_path, callback=self._refresh),
-                                os.path.dirname(self._collections_path))
+        self._observer.schedule(
+            _CollectionEventHandler(file_path=self._collections_path, callback=self._reload_and_reschedule),
+            os.path.dirname(self._collections_path))
         self._observer.start()
-        self._refresh()
+        self._reload_and_reschedule()
 
-    def collections(self) -> List[Collection]:
+    def collections(self) -> Set[Collection]:
         """
-        Return a list of all Collections being watched.
-        :return: A list of Collections
+        Return a set of all Collections being watched.
+        :return: A set of Collections
         """
-        return [collection for collections in self._collections_by_dir.values() for collection in collections]
+        return {collection for collections in self._collections_by_dir.values() for collection in collections}
+
+    def _validate_collection(self, collection: Collection):
+        directory = collection.directory()
+        if not os.path.isabs(directory):
+            raise RelativePathCollectionError(collection=collection)
+        if directory == os.path.dirname(self._collections_path):
+            raise ConflictingPathCollectionError(collection=collection)
 
     def _load_collections(self):
         try:
             with open(self._collections_path, 'r') as f:
                 collections_yaml = yaml.load(f, Loader=yaml.FullLoader)
             self._collections_by_dir.clear()
-            for _, collection_dict in collections_yaml.items():
-                collection = Collection.from_dict(collection_dict)
-                directory = collection.directory()
-                if directory == os.path.dirname(self._collections_path):
-                    logger.error(f"Collection {collection.dataset_id} uses granule directory {collection.path} "
-                                 f"which is the same directory as the collection configuration file, "
-                                 f"{self._collections_path}. The granules need to be in their own directory. "
-                                 f"Ignoring collection {collection.dataset_id} for now.")
-                else:
-                    self._collections_by_dir[directory].add(collection)
-
+            for collection_dict in collections_yaml['collections']:
+                try:
+                    collection = Collection.from_dict(collection_dict)
+                    self._validate_collection(collection)
+                    self._collections_by_dir[collection.directory()].add(collection)
+                except MissingValueCollectionError as e:
+                    logger.error(f"A collection is missing '{e.missing_value}'. Ignoring this collection for now.")
+                except RelativePathCollectionError as e:
+                    logger.error(f"Relative paths are not allowed for the 'path' property of a collection. "
+                                 f"Ignoring collection '{e.collection.dataset_id}' until its path is fixed.")
+                except ConflictingPathCollectionError as e:
+                    logger.error(f"Collection '{e.collection.dataset_id}' has granule path '{e.collection.path}' "
+                                 f"which uses same directory as the collection configuration file, "
+                                 f"'{self._collections_path}'. The granules need to be in their own directory. "
+                                 f"Ignoring collection '{e.collection.dataset_id}' for now.")
         except FileNotFoundError:
-            logger.error(f"Collection configuration file not found at {self._collections_path}.")
+            raise CollectionConfigFileNotFoundError("The collection config file could not be found at "
+                                                    f"{self._collections_path}")
         except yaml.scanner.ScannerError:
-            logger.error(f"Bad YAML syntax in collection configuration file. Will attempt to reload collections "
-                         f"after the next configuration change.")
-
-    def _refresh(self):
-        for collection in self._get_updated_collections():
-            self._collection_updated_callback(collection)
+            raise YamlParsingError("Bad YAML syntax in collection configuration file. Will attempt to reload "
+                                   "collections after the next configuration change.")
 
-        self._observer.unschedule_all()
-        self._schedule_watches()
-
-    def _get_updated_collections(self) -> List[Collection]:
+    def _get_updated_collections(self) -> Set[Collection]:
         old_collections = self.collections()
         self._load_collections()
-        return list(set(self.collections()) - set(old_collections))
+        return self.collections() - old_collections
+
+    def _reload_and_reschedule(self):
+        try:
+            for collection in self._get_updated_collections():
+                self._collection_updated_callback(collection)
+            self._unschedule_watches()
+            self._schedule_watches()
+        except YamlParsingError as e:
+            logger.error(e)
+
+    def _unschedule_watches(self):
+        for watch in self._granule_watches:
+            self._observer.unschedule(watch)
+        self._granule_watches.clear()
 
     def _schedule_watches(self):
         for directory, collections in self._collections_by_dir.items():
             granule_event_handler = _GranuleEventHandler(self._granule_updated_callback, collections)
             # Note: the Watchdog library does not schedule a new watch
             # if one is already scheduled for the same directory
-            self._observer.schedule(granule_event_handler, directory)
+            self._granule_watches.add(self._observer.schedule(granule_event_handler, directory))
 
 
 class _CollectionEventHandler(FileSystemEventHandler):
@@ -115,3 +143,12 @@ class _GranuleEventHandler(FileSystemEventHandler):
         for collection in self._collections_for_dir:
             if collection.owns_file(event.src_path):
                 self._callback(event.src_path, collection)
+
+    def on_modified(self, event):
+        super().on_modified(event)
+        if os.path.isdir(event.src_path):
+            return
+
+        for collection in self._collections_for_dir:
+            if collection.owns_file(event.src_path):
+                self._callback(event.src_path, collection)
diff --git a/collection_manager/tests/history_manager/__init__.py b/collection_manager/tests/entities/__init__.py
similarity index 100%
copy from collection_manager/tests/history_manager/__init__.py
copy to collection_manager/tests/entities/__init__.py
diff --git a/collection_manager/tests/entities/test_Collection.py b/collection_manager/tests/entities/test_Collection.py
new file mode 100644
index 0000000..46506d4
--- /dev/null
+++ b/collection_manager/tests/entities/test_Collection.py
@@ -0,0 +1,139 @@
+import os
+import unittest
+from datetime import datetime, timezone
+
+from collection_manager.entities import Collection
+from collection_manager.entities.exceptions import MissingValueCollectionError
+
+
+class TestCollection(unittest.TestCase):
+
+    def test_directory_with_directory(self):
+        directory = os.path.join(os.path.dirname(__file__), "../resources/data")
+        collection = Collection(dataset_id="test_dataset",
+                                path=directory,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        self.assertEqual(directory, collection.directory())
+
+    def test_directory_with_pattern(self):
+        pattern = os.path.join(os.path.dirname(__file__), "../resources/data/*.nc")
+        collection = Collection(dataset_id="test_dataset",
+                                path=pattern,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        self.assertEqual(os.path.dirname(pattern), collection.directory())
+
+    def test_owns_file_raises_exception_with_directory(self):
+        directory = os.path.join(os.path.dirname(__file__), "../resources/data")
+        collection = Collection(dataset_id="test_dataset",
+                                path=directory,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        self.assertRaises(IsADirectoryError, collection.owns_file, directory)
+
+    def test_owns_file_matches(self):
+        directory = os.path.join(os.path.dirname(__file__), "../resources/data")
+        collection = Collection(dataset_id="test_dataset",
+                                path=directory,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        file_path = os.path.join(directory, "test_file.nc")
+        self.assertTrue(collection.owns_file(file_path))
+
+    def test_owns_file_does_not_match(self):
+        directory = os.path.join(os.path.dirname(__file__), "../resources/data")
+        collection = Collection(dataset_id="test_dataset",
+                                path=directory,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        self.assertFalse(collection.owns_file("test_file.nc"))
+
+    def test_owns_file_matches_pattern(self):
+        directory = os.path.join(os.path.dirname(__file__), "../resources/data")
+        pattern = os.path.join(directory, "test_*.nc")
+        collection = Collection(dataset_id="test_dataset",
+                                path=pattern,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        file_path = os.path.join(directory, "test_file.nc")
+        self.assertTrue(collection.owns_file(file_path))
+
+    def test_owns_file_does_not_match_pattern(self):
+        directory = os.path.join(os.path.dirname(__file__), "../resources/data")
+        pattern = os.path.join(directory, "test_*.nc")
+        collection = Collection(dataset_id="test_dataset",
+                                path=pattern,
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        file_path = os.path.join(directory, "nonmatch.nc")
+        self.assertFalse(collection.owns_file(file_path))
+
+    def test_from_dict(self):
+        collection_dict = {
+            'id': 'test_id',
+            'variable': 'test_var',
+            'path': '/some/path',
+            'priority': 1,
+            'forward-processing-priority': 2,
+            'from': '2020-01-01T00:00:00+00:00',
+            'to': '2020-02-01T00:00:00+00:00'
+        }
+
+        expected_collection = Collection(dataset_id='test_id',
+                                         variable='test_var',
+                                         path='/some/path',
+                                         historical_priority=1,
+                                         forward_processing_priority=2,
+                                         date_from=datetime(2020, 1, 1, 0, 0, 0, tzinfo=timezone.utc),
+                                         date_to=datetime(2020, 2, 1, 0, 0, 0, tzinfo=timezone.utc))
+
+        self.assertEqual(expected_collection, Collection.from_dict(collection_dict))
+
+    def test_from_dict_missing_optional_values(self):
+        collection_dict = {
+            'id': 'test_id',
+            'variable': 'test_var',
+            'path': '/some/path',
+            'priority': 3
+        }
+
+        expected_collection = Collection(dataset_id='test_id',
+                                         variable='test_var',
+                                         path='/some/path',
+                                         historical_priority=3,
+                                         forward_processing_priority=None,
+                                         date_from=None,
+                                         date_to=None)
+
+        self.assertEqual(expected_collection, Collection.from_dict(collection_dict))
+
+    def test_from_dict_missing_required_values(self):
+        collection_dict = {
+            'id': 'test_id',
+            'variable': 'test_var',
+            'path': '/some/path',
+        }
+
+        self.assertRaises(MissingValueCollectionError, Collection.from_dict, collection_dict)
diff --git a/collection_manager/tests/history_manager/test_datasetingestionhistoryfile.py b/collection_manager/tests/history_manager/test_datasetingestionhistoryfile.py
deleted file mode 100644
index 0edeafb..0000000
--- a/collection_manager/tests/history_manager/test_datasetingestionhistoryfile.py
+++ /dev/null
@@ -1,64 +0,0 @@
-import unittest
-import os
-import sys
-import pathlib
-from collection_manager.services.history_manager import FileIngestionHistory
-from collection_manager.services.history_manager import md5sum_from_filepath
-
-HISTORY_ROOT_PATH = os.path.join(sys.prefix,
-                                 ".collection_manager",
-                                 "tmp/history")
-DATASET_ID = "zobi_la_mouche"
-
-
-class DatasetIngestionHistoryFileTestCase(unittest.TestCase):
-    ingestion_history = None
-
-    # @unittest.skip("does not work without a solr server for history_manager")
-    def test_get_md5sum(self):
-        self.ingestion_history = FileIngestionHistory(HISTORY_ROOT_PATH, DATASET_ID, md5sum_from_filepath)
-        self.ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
-        result = self.ingestion_history._get_signature("blue")
-        self.assertEqual(result, "12weeukrhbwerqu7wier")
-
-    # @unittest.skip("does not work without a solr server for history_manager")
-    def test_get_missing_md5sum(self):
-        self.ingestion_history = FileIngestionHistory(HISTORY_ROOT_PATH, DATASET_ID, md5sum_from_filepath)
-        self.ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
-        result = self.ingestion_history._get_signature("green")
-        self.assertEqual(result, None)
-
-    def test_has_valid_cache(self):
-        self.ingestion_history = FileIngestionHistory(HISTORY_ROOT_PATH, DATASET_ID, md5sum_from_filepath)
-        # history_manager with this file
-        current_file_path = pathlib.Path(__file__)
-        self.ingestion_history.push(str(current_file_path))
-        self.assertEqual(self.ingestion_history.already_ingested(str(current_file_path)), True)
-
-    def test_has_valid_cache_with_latest_modifcation_signature(self):
-        self.ingestion_history = FileIngestionHistory(HISTORY_ROOT_PATH, DATASET_ID, os.path.getmtime)
-        # history_manager with this file
-        current_file_path = pathlib.Path(__file__)
-        self.ingestion_history.push(str(current_file_path))
-        self.assertEqual(self.ingestion_history.already_ingested(str(current_file_path)), True)
-
-    def test_has_not_valid_cache(self):
-        self.ingestion_history = FileIngestionHistory(HISTORY_ROOT_PATH, DATASET_ID, md5sum_from_filepath)
-        # history_manager with this file
-        current_file_path = pathlib.Path(__file__)
-        self.assertEqual(self.ingestion_history.already_ingested(str(current_file_path)), False)
-
-    @unittest.skip("skip before history_manager dataset is not available")
-    def test_purge(self):
-        self.ingestion_history = FileIngestionHistory("/Users/loubrieu/PycharmProjects/collection_manager/venv/.collection_manager/tmp/history/",
-                                                             "avhrr-oi-analysed-sst-toto",
-                                                      lambda x : str(os.path.getmtime(x)))
-        self.ingestion_history.purge()
-
-    def tearDown(self):
-        self.ingestion_history.reset_cache()
-        del self.ingestion_history
-
-
-if __name__ == '__main__':
-    unittest.main()
diff --git a/collection_manager/tests/resources/collections.yml b/collection_manager/tests/resources/collections.yml
new file mode 100644
index 0000000..89524ec
--- /dev/null
+++ b/collection_manager/tests/resources/collections.yml
@@ -0,0 +1,17 @@
+collections:
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+    path: /opt/data/grace/*land*.nc
+    variable: lwe_thickness
+    priority: 1
+    forward-processing-priority: 5
+
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_OCEAN
+    path: /opt/data/grace/*ocean*.nc
+    variable: lwe_thickness
+    priority: 2
+    forward-processing-priority: 6
+
+  - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
+    path: /opt/data/avhrr/*.nc
+    variable: analysed_sst
+    priority: 1
diff --git a/collection_manager/tests/resources/collections_alternate.yml b/collection_manager/tests/resources/collections_alternate.yml
new file mode 100644
index 0000000..3d7da95
--- /dev/null
+++ b/collection_manager/tests/resources/collections_alternate.yml
@@ -0,0 +1,17 @@
+collections:
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+    path: /opt/data/grace/*land*.nc
+    variable: lwe_thickness
+    priority: 1
+    forward-processing-priority: 5
+
+  - id: ID_CHANGED
+    path: /opt/data/grace/*ocean*.nc
+    variable: lwe_thickness
+    priority: 2
+    forward-processing-priority: 6
+
+  - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
+    path: /opt/data/avhrr/*.nc
+    variable: analysed_sst
+    priority: 1
diff --git a/collection_manager/tests/resources/collections_bad.yml b/collection_manager/tests/resources/collections_bad.yml
new file mode 100644
index 0000000..cac6a32
--- /dev/null
+++ b/collection_manager/tests/resources/collections_bad.yml
@@ -0,0 +1,17 @@
+collections:
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+    path: /opt/data/grace/*land*.nc
+    variable: lwe_thickness
+    priority: 1
+    forward-processing-priority: 5
+BAD SYNTAX!
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_OCEAN
+    path: /opt/data/grace/*ocean*.nc
+    variable: lwe_thickness
+    priority: 2
+    forward-processing-priority: 6
+
+  - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
+    path: /opt/data/avhrr/*.nc
+    variable: analysed_sst
+    priority: 1
diff --git a/collection_manager/tests/resources/data/collections.yml b/collection_manager/tests/resources/data/collections.yml
deleted file mode 100644
index 8c30a37..0000000
--- a/collection_manager/tests/resources/data/collections.yml
+++ /dev/null
@@ -1,11 +0,0 @@
-avhrr-oi-analysed-sst:
-  id: avhrr-oi-analysed-sst
-  path: ../tests/resources/data/avhrr_oi/*.nc
-  variable: analysed_sst
-  priority: 9
-
-avhrr-oi-analysed-sst2:
-  id: avhrr-oi-analysed-sst2
-  path: ../tests/resources/data/avhrr_oi/*.nc
-  variable: analysed_sst
-  priority: 1
diff --git a/collection_manager/tests/resources/data/dataset_config_file_ok.yml b/collection_manager/tests/resources/data/dataset_config_file_ok.yml
deleted file mode 100644
index 66bb883..0000000
--- a/collection_manager/tests/resources/data/dataset_config_file_ok.yml
+++ /dev/null
@@ -1,44 +0,0 @@
-# Tile Slicer Config
-ningester:
-    tile_slicer: sliceFileByTilesDesired
-    sliceFileByTilesDesired:
-      tilesDesired: 1296
-      timeDimension: time
-      dimensions:
-        - lat
-        - lon
----
-# Tile processors configuration
-ningester:
-  tile_processors:
-    - pythonChainProcessor
-    - generateTileId
-    - addDatasetName
-  pythonChainProcessor:
-    enabled:
-    processor_list:
-      -
-        name: GridReadingProcessor
-        config:
-          latitude: lat
-          longitude: lon
-          time: time
-          variable_to_read: analysed_sst
-      -
-        name: EmptyTileFilter
-      -
-        name: TileSummarizingProcessor
-        config:
-          stored_var_name: analysed_sst
-  generateTileId:
-    enabled:
-    salt: analysed_sst
-  addDatasetName:
-    enabled:
-    datasetName: avhrr-oi-analysed-sst
----
-# Tile writer configuration
-ningester:
-  tile_writer:
-    data_store: cassandraStore
-    metadata_store: solrStore
diff --git a/collection_manager/tests/history_manager/__init__.py b/collection_manager/tests/services/history_manager/__init__.py
similarity index 100%
rename from collection_manager/tests/history_manager/__init__.py
rename to collection_manager/tests/services/history_manager/__init__.py
diff --git a/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py b/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
new file mode 100644
index 0000000..d2ad45c
--- /dev/null
+++ b/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
@@ -0,0 +1,56 @@
+import os
+import pathlib
+import tempfile
+import unittest
+
+from collection_manager.services.history_manager import FileIngestionHistory, md5sum_from_filepath
+
+DATASET_ID = "zobi_la_mouche"
+
+
+class TestFileIngestionHistory(unittest.TestCase):
+
+    def test_get_md5sum(self):
+        with tempfile.TemporaryDirectory() as history_dir:
+            ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
+            ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
+            result = ingestion_history._get_signature("blue")
+            self.assertEqual(result, "12weeukrhbwerqu7wier")
+
+    def test_get_missing_md5sum(self):
+        with tempfile.TemporaryDirectory() as history_dir:
+            ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
+            ingestion_history._push_record("blue", "12weeukrhbwerqu7wier")
+            result = ingestion_history._get_signature("green")
+            self.assertIsNone(result)
+
+    def test_already_ingested(self):
+        with tempfile.TemporaryDirectory() as history_dir:
+            ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
+            # history_manager with this file
+            current_file_path = pathlib.Path(__file__)
+            ingestion_history.push(str(current_file_path))
+            self.assertTrue(ingestion_history.already_ingested(str(current_file_path)))
+
+            del ingestion_history
+
+    def test_already_ingested_with_latest_modifcation_signature(self):
+        with tempfile.TemporaryDirectory() as history_dir:
+            ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, os.path.getmtime)
+            # history_manager with this file
+            current_file_path = pathlib.Path(__file__)
+            ingestion_history.push(str(current_file_path))
+            self.assertTrue(ingestion_history.already_ingested(str(current_file_path)))
+
+            del ingestion_history
+
+    def test_already_ingested_is_false(self):
+        with tempfile.TemporaryDirectory() as history_dir:
+            ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
+            # history_manager with this file
+            current_file_path = pathlib.Path(__file__)
+            self.assertFalse(ingestion_history.already_ingested(str(current_file_path)))
+
+
+if __name__ == '__main__':
+    unittest.main()
diff --git a/collection_manager/tests/history_manager/test_datasetingestionhistorysolr.py b/collection_manager/tests/services/history_manager/test_SolrIngestionHistory.py
similarity index 94%
rename from collection_manager/tests/history_manager/test_datasetingestionhistorysolr.py
rename to collection_manager/tests/services/history_manager/test_SolrIngestionHistory.py
index 57b2bd1..deab42d 100644
--- a/collection_manager/tests/history_manager/test_datasetingestionhistorysolr.py
+++ b/collection_manager/tests/services/history_manager/test_SolrIngestionHistory.py
@@ -5,7 +5,8 @@ SOLR_URL = "http://localhost:8984/solr"
 DATASET_ID = "zobi_la_mouche"
 
 
-class DatasetIngestionHistorySolrTestCase(unittest.TestCase):
+# TODO: mock solr and fix these tests
+class TestSolrIngestionHistory(unittest.TestCase):
     @unittest.skip("does not work without a solr server for history_manager")
     def test_get(self):
         ingestion_history = SolrIngestionHistory(SOLR_URL, DATASET_ID)
diff --git a/collection_manager/tests/services/test_CollectionProcessor.py b/collection_manager/tests/services/test_CollectionProcessor.py
index 73b67ba..7899e22 100644
--- a/collection_manager/tests/services/test_CollectionProcessor.py
+++ b/collection_manager/tests/services/test_CollectionProcessor.py
@@ -1,10 +1,11 @@
 import tempfile
-from unittest import mock
 import unittest
+from unittest import mock
 
 from collection_manager.entities import Collection
 from collection_manager.services import CollectionProcessor
-from collection_manager.services.history_manager import FileIngestionHistoryBuilder, FileIngestionHistory
+from collection_manager.services.history_manager import FileIngestionHistoryBuilder
+from collection_manager.services.history_manager import GranuleStatus
 
 
 class TestCollectionProcessor(unittest.TestCase):
@@ -18,15 +19,17 @@ class TestCollectionProcessor(unittest.TestCase):
     def test_file_supported_with_foo(self):
         self.assertFalse(CollectionProcessor._file_supported("test_dir/test_granule.foo"))
 
-    def test_get_history_manager_returns_same_object(self):
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_get_history_manager_returns_same_object(self, mock_publisher):
         with tempfile.TemporaryDirectory() as history_dir:
-            collection_processor = CollectionProcessor(None, FileIngestionHistoryBuilder(history_dir))
+            collection_processor = CollectionProcessor(mock_publisher, FileIngestionHistoryBuilder(history_dir))
             history_manager = collection_processor._get_history_manager('dataset_id')
             self.assertIs(collection_processor._get_history_manager('dataset_id'), history_manager)
 
-    def test_get_history_manager_returns_different_object(self):
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_get_history_manager_returns_different_object(self, mock_publisher):
         with tempfile.TemporaryDirectory() as history_dir:
-            collection_processor = CollectionProcessor(None, FileIngestionHistoryBuilder(history_dir))
+            collection_processor = CollectionProcessor(mock_publisher, FileIngestionHistoryBuilder(history_dir))
             history_manager = collection_processor._get_history_manager('foo')
             self.assertIsNot(collection_processor._get_history_manager('bar'), history_manager)
 
@@ -60,7 +63,106 @@ class TestCollectionProcessor(unittest.TestCase):
         filled = CollectionProcessor._fill_template(collection, template)
         self.assertEqual(filled, expected)
 
-    @mock.patch.object(FileIngestionHistory, 'push')
-    @mock.patch.object(FileIngestionHistory, 'get_granule_status')
-    def test_process_granule(self):
-        history_manager_builder = FileIngestionHistoryBuilder('/foo')
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_process_granule_with_historical_granule(self, mock_publisher, mock_history_builder, mock_history):
+        mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_HISTORICAL
+        mock_history_builder.build.return_value = mock_history
+
+        collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
+        collection = Collection(dataset_id="test_dataset",
+                                path="test_path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+
+        collection_processor.process_granule("test.nc", collection)
+
+        mock_publisher.publish_message.assert_called_with(body=mock.ANY, priority=1)
+        mock_history.push.assert_called()
+
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_process_granule_with_forward_processing_granule(self, mock_publisher, mock_history_builder, mock_history):
+        mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_FORWARD_PROCESSING
+        mock_history_builder.build.return_value = mock_history
+
+        collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
+        collection = Collection(dataset_id="test_dataset",
+                                path="test_path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+
+        collection_processor.process_granule("test.h5", collection)
+
+        mock_publisher.publish_message.assert_called_with(body=mock.ANY, priority=2)
+        mock_history.push.assert_called()
+
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_process_granule_with_forward_processing_granule_and_no_priority(self, mock_publisher,
+                                                                             mock_history_builder, mock_history):
+        mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_FORWARD_PROCESSING
+        mock_history_builder.build.return_value = mock_history
+
+        collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
+        collection = Collection(dataset_id="test_dataset",
+                                path="test_path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                date_from=None,
+                                date_to=None)
+
+        collection_processor.process_granule("test.h5", collection)
+
+        mock_publisher.publish_message.assert_called_with(body=mock.ANY, priority=1)
+        mock_history.push.assert_called()
+
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_process_granule_with_undesired_granule(self, mock_publisher, mock_history_builder, mock_history):
+        mock_history.get_granule_status.return_value = GranuleStatus.UNDESIRED
+        mock_history_builder.build.return_value = mock_history
+
+        collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
+        collection = Collection(dataset_id="test_dataset",
+                                path="test_path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+
+        collection_processor.process_granule("test.nc", collection)
+
+        mock_publisher.publish_message.assert_not_called()
+        mock_history.push.assert_not_called()
+
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
+    @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
+    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
+    def test_process_granule_with_unsupported_file_type(self, mock_publisher, mock_history_builder, mock_history):
+        mock_history_builder.build.return_value = mock_history
+
+        collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
+        collection = Collection(dataset_id="test_dataset",
+                                path="test_path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+
+        collection_processor.process_granule("test.foo", collection)
+
+        mock_publisher.publish_message.assert_not_called()
+        mock_history.push.assert_not_called()
diff --git a/collection_manager/tests/services/test_CollectionWatcher.py b/collection_manager/tests/services/test_CollectionWatcher.py
new file mode 100644
index 0000000..7ae25a1
--- /dev/null
+++ b/collection_manager/tests/services/test_CollectionWatcher.py
@@ -0,0 +1,203 @@
+import os
+import tempfile
+import time
+import unittest
+from datetime import datetime
+from unittest.mock import Mock
+
+from collection_manager.entities import Collection
+from collection_manager.entities.exceptions import YamlParsingError, CollectionConfigFileNotFoundError, \
+    RelativePathCollectionError, ConflictingPathCollectionError
+from collection_manager.services import CollectionWatcher
+
+
+class TestCollectionWatcher(unittest.TestCase):
+
+    def test_collections_returns_all_collections(self):
+        collection_watcher = CollectionWatcher('/foo', Mock(), Mock())
+        collection_watcher._collections_by_dir = {
+            "/foo": {
+                Collection("id1", "var1", "path1", 1, 2, datetime.now(), datetime.now()),
+                Collection("id2", "var2", "path2", 3, 4, datetime.now(), datetime.now()),
+            },
+            "/bar": {
+                Collection("id3", "var3", "path3", 5, 6, datetime.now(), datetime.now()),
+                Collection("id4", "var4", "path4", 7, 8, datetime.now(), datetime.now()),
+            }
+        }
+        flattened_collections = collection_watcher.collections()
+        self.assertEqual(len(flattened_collections), 4)
+
+    def test_load_collections_loads_all_collections(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+        collection_watcher._load_collections()
+
+        self.assertEqual(len(collection_watcher._collections_by_dir), 2)
+        self.assertEqual(len(collection_watcher._collections_by_dir['/opt/data/grace']), 2)
+        self.assertEqual(len(collection_watcher._collections_by_dir['/opt/data/avhrr']), 1)
+
+    def test_load_collections_with_bad_yaml_syntax(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections_bad.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        self.assertRaises(YamlParsingError, collection_watcher._load_collections)
+
+    def test_load_collections_with_file_not_found(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/does_not_exist.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        self.assertRaises(CollectionConfigFileNotFoundError, collection_watcher._load_collections)
+
+    def test_get_updated_collections_returns_all_collections(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        updated_collections = collection_watcher._get_updated_collections()
+        self.assertSetEqual(updated_collections, collection_watcher.collections())
+
+    def test_get_updated_collections_returns_no_collections(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+        collection_watcher._load_collections()
+        updated_collections = collection_watcher._get_updated_collections()
+
+        self.assertEqual(len(updated_collections), 0)
+
+    def test_get_updated_collections_returns_some_collections(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+        collection_watcher._load_collections()
+
+        new_collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections_alternate.yml')
+        collection_watcher._collections_path = new_collections_path
+        updated_collections = collection_watcher._get_updated_collections()
+
+        self.assertEqual(len(updated_collections), 1)
+
+    def test_validate_collection(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        collection = Collection(dataset_id="test_dataset",
+                                path="/absolute/path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        collection_watcher._validate_collection(collection)
+
+    def test_validate_collection_with_relative_path(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        collection = Collection(dataset_id="test_dataset",
+                                path="relative/path",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        self.assertRaises(RelativePathCollectionError, collection_watcher._validate_collection, collection)
+
+    def test_validate_collection_with_conflicting_path(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '/resources/collections.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        collection = Collection(dataset_id="test_dataset",
+                                path="/resources/*.nc",
+                                variable="test_variable",
+                                historical_priority=1,
+                                forward_processing_priority=2,
+                                date_from=None,
+                                date_to=None)
+        self.assertRaises(ConflictingPathCollectionError, collection_watcher._validate_collection, collection)
+
+    def test_collection_callback_is_called(self):
+        collections_config = tempfile.NamedTemporaryFile("w+b", buffering=0, delete=False)
+        granule_dir = tempfile.TemporaryDirectory()
+        collections_str = f"""collections:
+- id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+  path: {granule_dir.name}
+  variable: lwe_thickness
+  priority: 1
+  forward-processing-priority: 5"""
+        collections_config.write(collections_str.encode("utf-8"))
+
+        collection_callback = Mock()
+        collection_watcher = CollectionWatcher(collections_config.name, collection_callback, Mock())
+        collection_watcher.start_watching()
+
+        collections_str = f"""
+- id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+  path: {granule_dir.name}
+  variable: lwe_thickness
+  priority: 10
+  forward-processing-priority: 5
+        """
+        collections_config.write(collections_str.encode("utf-8"))
+        collections_config.close()
+
+        self.assert_called_within_timeout(collection_callback, timeout_sec=1, call_count=2)
+        granule_dir.cleanup()
+        os.remove(collections_config.name)
+
+    def test_granule_callback_is_called_on_new_file(self):
+        with tempfile.NamedTemporaryFile("w+b", buffering=0) as collections_config:
+            granule_dir = tempfile.TemporaryDirectory()
+            collections_str = f"""
+collections:
+- id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+  path: {granule_dir.name}
+  variable: lwe_thickness
+  priority: 1
+  forward-processing-priority: 5
+            """
+            collections_config.write(collections_str.encode("utf-8"))
+
+            granule_callback = Mock()
+            collection_watcher = CollectionWatcher(collections_config.name, Mock(), granule_callback)
+            collection_watcher.start_watching()
+
+            new_granule = open(os.path.join(granule_dir.name, 'test.nc'), "w+")
+
+            self.assert_called_within_timeout(granule_callback)
+
+            new_granule.close()
+            granule_dir.cleanup()
+
+    def test_granule_callback_is_called_on_modified_file(self):
+        with tempfile.NamedTemporaryFile("w+b", buffering=0) as collections_config:
+            granule_dir = tempfile.TemporaryDirectory()
+            collections_str = f"""
+collections:
+- id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
+  path: {granule_dir.name}
+  variable: lwe_thickness
+  priority: 1
+  forward-processing-priority: 5
+            """
+            collections_config.write(collections_str.encode("utf-8"))
+            new_granule = open(os.path.join(granule_dir.name, 'test.nc'), "w+")
+
+            granule_callback = Mock()
+            collection_watcher = CollectionWatcher(collections_config.name, Mock(), granule_callback)
+            collection_watcher.start_watching()
+
+            new_granule.write("hello world")
+            new_granule.close()
+
+            self.assert_called_within_timeout(granule_callback)
+
+            granule_dir.cleanup()
+
+    @staticmethod
+    def assert_called_within_timeout(mock_func, timeout_sec=1.0, call_count=1):
+        start = datetime.now()
+
+        while (datetime.now() - start).total_seconds() < timeout_sec:
+            time.sleep(0.01)
+            if mock_func.call_count >= call_count:
+                return
+        raise AssertionError(f"{mock_func} did not reach {call_count} calls called within {timeout_sec} sec")

[incubator-sdap-ingester] 08/33: SDAP-270: Fix bad solr endpoints in solr history manager (#8)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit bf8c911f809428542296def9bf74c27c913d0faa
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Mon Aug 3 19:06:18 2020 -0700

    SDAP-270: Fix bad solr endpoints in solr history manager (#8)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 .../history_manager/SolrIngestionHistory.py        | 40 ++++++++++------------
 1 file changed, 18 insertions(+), 22 deletions(-)

diff --git a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
index 1ae7156..319fa3e 100644
--- a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
@@ -33,10 +33,10 @@ class SolrIngestionHistory(IngestionHistory):
 
     def __init__(self, solr_url: str, dataset_id: str, signature_fun=None):
         try:
-            self._solr_url = solr_url
+            self._url_prefix = f"{solr_url.strip('/')}/solr"
             self._create_collection_if_needed()
-            self._solr_granules = pysolr.Solr('/'.join([solr_url.strip('/'), self._granule_collection_name]))
-            self._solr_datasets = pysolr.Solr('/'.join([solr_url.strip('/'), self._dataset_collection_name]))
+            self._solr_granules = pysolr.Solr(f"{self._url_prefix}/{self._granule_collection_name}")
+            self._solr_datasets = pysolr.Solr(f"{self._url_prefix}/{self._dataset_collection_name}")
             self._dataset_id = dataset_id
             self._signature_fun = md5sum_from_filepath if signature_fun is None else signature_fun
             self._latest_ingested_file_update = self._get_latest_file_update()
@@ -63,7 +63,7 @@ class SolrIngestionHistory(IngestionHistory):
             self._solr_datasets.add([{
                 'id': self._dataset_id,
                 'dataset_s': self._dataset_id,
-                'latest_update_l': self._latest_ingested_file_update}])
+                'latest_update_l': int(self._latest_ingested_file_update)}])
             self._solr_datasets.commit()
 
     def _get_latest_file_update(self):
@@ -87,8 +87,8 @@ class SolrIngestionHistory(IngestionHistory):
                 self._req_session = requests.session()
 
             payload = {'action': 'CLUSTERSTATUS'}
-            result = self._req_session.get('/'.join([self._solr_url.strip('/'), 'admin', 'collections']),
-                                           params=payload)
+            collections_endpoint = f"{self._url_prefix}/admin/collections"
+            result = self._req_session.get(collections_endpoint, params=payload)
             response = result.json()
             node_number = len(response['cluster']['live_nodes'])
 
@@ -100,17 +100,15 @@ class SolrIngestionHistory(IngestionHistory):
                            'name': self._granule_collection_name,
                            'numShards': node_number
                            }
-                result = self._req_session.get('/'.join([self._solr_url.strip("/"), 'admin', 'collections']),
-                                               params=payload)
+                result = self._req_session.get(collections_endpoint, params=payload)
                 response = result.json()
                 logger.info(f"solr collection created {response}")
+
                 # Update schema
-                schema_url = '/'.join([self._solr_url.strip('/'), self._granule_collection_name, 'schema'])
-                # granule_s # dataset_s so that all the granule of a dataset are less likely to be on the same shard
-                # self.add_unique_key_field(schema_url, "uniqueKey_s", "StrField")
-                self._add_field(schema_url, "dataset_s", "StrField")
-                self._add_field(schema_url, "granule_s", "StrField")
-                self._add_field(schema_url, "granule_signature_s", "StrField")
+                schema_endpoint = f"{self._url_prefix/{self._granule_collection_name}/schema"
+                self._add_field(schema_endpoint, "dataset_s", "string")
+                self._add_field(schema_endpoint, "granule_s", "string")
+                self._add_field(schema_endpoint, "granule_signature_s", "string")
 
             else:
                 logger.info(f"collection {self._granule_collection_name} already exists")
@@ -121,16 +119,14 @@ class SolrIngestionHistory(IngestionHistory):
                            'name': self._dataset_collection_name,
                            'numShards': node_number
                            }
-                result = self._req_session.get('/'.join([self._solr_url.strip('/'), 'admin', 'collections']),
-                                               params=payload)
+                result = self._req_session.get(collections_endpoint, params=payload)
                 response = result.json()
                 logger.info(f"solr collection created {response}")
+
                 # Update schema
-                # http://localhost:8983/solr/nexusdatasets/schema?_=1588555874864&wt=json
-                schema_url = '/'.join([self._solr_url.strip('/'), self._dataset_collection_name, 'schema'])
-                # self.add_unique_key_field(schema_url, "uniqueKey_s", "StrField")
-                self._add_field(schema_url, "dataset_s", "StrField")
-                self._add_field(schema_url, "latest_update_l", "TrieLongField")
+                schema_endpoint = f"{self._url_prefix}/{self._dataset_collection_name}/schema"
+                self._add_field(schema_endpoint, "dataset_s", "string")
+                self._add_field(schema_endpoint, "latest_update_l", "TrieLongField")
 
             else:
                 logger.info(f"collection {self._dataset_collection_name} already exists")
@@ -154,7 +150,7 @@ class SolrIngestionHistory(IngestionHistory):
                 "stored": False
             }
         }
-        result = self._req_session.post(schema_url, data=add_field_payload.__str__())
+        return self._req_session.post(schema_url, data=str(add_field_payload).encode('utf-8'))
 
 
 class DatasetIngestionHistorySolrException(Exception):

[incubator-sdap-ingester] 02/33: SDAP-254, SDAP-255, SDAP-256: Fix bug where ingestion history is not saved, fix bug where messages published to RabbitMQ are incorrect, fix bug where bad collection config file crashes app (#3)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 2c991c5edd260c57e2402664f79a15b77acf6967
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Mon Jun 22 16:51:58 2020 -0700

    SDAP-254, SDAP-255, SDAP-256: Fix bug where ingestion history is not saved, fix bug where messages published to RabbitMQ are incorrect, fix bug where bad collection config file crashes app (#3)
---
 .../entities/exceptions/Exceptions.py                  |  2 +-
 .../collection_manager/entities/exceptions/__init__.py |  2 +-
 collection_manager/collection_manager/main.py          |  6 +++++-
 .../collection_manager/services/CollectionProcessor.py |  6 +++---
 .../collection_manager/services/CollectionWatcher.py   | 18 +++++++++++++-----
 .../services/history_manager/FileIngestionHistory.py   |  4 ++--
 .../services/history_manager/IngestionHistory.py       |  6 ++++++
 .../services/history_manager/SolrIngestionHistory.py   |  3 +--
 ...{collections_bad.yml => collections_bad_schema.yml} |  4 ++--
 ...{collections_bad.yml => collections_bad_syntax.yml} |  0
 .../tests/services/test_CollectionProcessor.py         |  6 +++---
 .../tests/services/test_CollectionWatcher.py           | 12 +++++++++---
 12 files changed, 46 insertions(+), 23 deletions(-)

diff --git a/collection_manager/collection_manager/entities/exceptions/Exceptions.py b/collection_manager/collection_manager/entities/exceptions/Exceptions.py
index 8e63d24..c18c4c8 100644
--- a/collection_manager/collection_manager/entities/exceptions/Exceptions.py
+++ b/collection_manager/collection_manager/entities/exceptions/Exceptions.py
@@ -2,7 +2,7 @@ class RelativePathError(Exception):
     pass
 
 
-class YamlParsingError(Exception):
+class CollectionConfigParsingError(Exception):
     pass
 
 
diff --git a/collection_manager/collection_manager/entities/exceptions/__init__.py b/collection_manager/collection_manager/entities/exceptions/__init__.py
index 9a22c16..7fac507 100644
--- a/collection_manager/collection_manager/entities/exceptions/__init__.py
+++ b/collection_manager/collection_manager/entities/exceptions/__init__.py
@@ -3,4 +3,4 @@ from .Exceptions import ConflictingPathCollectionError
 from .Exceptions import MissingValueCollectionError
 from .Exceptions import RelativePathCollectionError
 from .Exceptions import RelativePathError
-from .Exceptions import YamlParsingError
+from .Exceptions import CollectionConfigParsingError
diff --git a/collection_manager/collection_manager/main.py b/collection_manager/collection_manager/main.py
index a10446f..bc2d356 100644
--- a/collection_manager/collection_manager/main.py
+++ b/collection_manager/collection_manager/main.py
@@ -70,8 +70,12 @@ def main():
                                                granule_updated_callback=collection_processor.process_granule)
 
         collection_watcher.start_watching()
+
         while True:
-            time.sleep(1)
+            try:
+                time.sleep(1)
+            except KeyboardInterrupt:
+                return
 
     except Exception as e:
         logger.error(e)
diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index a81390b..232cdee 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -63,7 +63,7 @@ class CollectionProcessor:
                         f"time range for collection '{collection.dataset_id}'. Skipping.")
             return
 
-        dataset_config = self._fill_template(collection, config_template=self._config_template)
+        dataset_config = self._fill_template(granule, collection, config_template=self._config_template)
         self._publisher.publish_message(body=dataset_config, priority=use_priority)
         history_manager.push(granule)
 
@@ -78,11 +78,11 @@ class CollectionProcessor:
         return self._history_manager_cache[dataset_id]
 
     @staticmethod
-    def _fill_template(collection: Collection, config_template: str) -> str:
+    def _fill_template(granule_path: str, collection: Collection, config_template: str) -> str:
         renderer = pystache.Renderer()
         config_content = renderer.render(config_template,
                                          {
-                                             'granule': collection.path,
+                                             'granule': granule_path,
                                              'dataset_id': collection.dataset_id,
                                              'variable': collection.variable
                                          })
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index 6bbe7d9..a3c3bf7 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -9,7 +9,7 @@ from watchdog.observers import Observer
 from yaml.scanner import ScannerError
 
 from collection_manager.entities import Collection
-from collection_manager.entities.exceptions import RelativePathError, YamlParsingError, \
+from collection_manager.entities.exceptions import RelativePathError, CollectionConfigParsingError, \
     CollectionConfigFileNotFoundError, MissingValueCollectionError, ConflictingPathCollectionError, \
     RelativePathCollectionError
 
@@ -84,8 +84,12 @@ class CollectionWatcher:
             raise CollectionConfigFileNotFoundError("The collection config file could not be found at "
                                                     f"{self._collections_path}")
         except yaml.scanner.ScannerError:
-            raise YamlParsingError("Bad YAML syntax in collection configuration file. Will attempt to reload "
-                                   "collections after the next configuration change.")
+            raise CollectionConfigParsingError("Bad YAML syntax in collection configuration file. Will attempt "
+                                               "to reload collections after the next configuration change.")
+        except KeyError:
+            raise CollectionConfigParsingError("The collections configuration YAML file does not conform to the "
+                                               "proper schema. Will attempt to reload collections config after the "
+                                               "next file modification.")
 
     def _get_updated_collections(self) -> Set[Collection]:
         old_collections = self.collections()
@@ -98,7 +102,7 @@ class CollectionWatcher:
                 self._collection_updated_callback(collection)
             self._unschedule_watches()
             self._schedule_watches()
-        except YamlParsingError as e:
+        except CollectionConfigParsingError as e:
             logger.error(e)
 
     def _unschedule_watches(self):
@@ -111,7 +115,11 @@ class CollectionWatcher:
             granule_event_handler = _GranuleEventHandler(self._granule_updated_callback, collections)
             # Note: the Watchdog library does not schedule a new watch
             # if one is already scheduled for the same directory
-            self._granule_watches.add(self._observer.schedule(granule_event_handler, directory))
+            try:
+                self._granule_watches.add(self._observer.schedule(granule_event_handler, directory))
+            except (FileNotFoundError, NotADirectoryError):
+                bad_collection_names = ' and '.join([col.dataset_id for col in collections])
+                logger.error(f"Granule directory {directory} does not exist. Ignoring {bad_collection_names}.")
 
 
 class _CollectionEventHandler(FileSystemEventHandler):
diff --git a/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
index 0a92317..50f2170 100644
--- a/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
@@ -20,6 +20,7 @@ class FileIngestionHistoryBuilder(IngestionHistoryBuilder):
                                     signature_fun=self._signature_fun)
 
 
+# TODO: clean this up, better tests
 class FileIngestionHistory(IngestionHistory):
 
     def __init__(self, history_path: str, dataset_id: str, signature_fun=None):
@@ -55,7 +56,6 @@ class FileIngestionHistory(IngestionHistory):
     def __del__(self):
         self._history_file.close()
         self._purge()
-        self._save_latest_timestamp()
         del self._history_dict
 
     def reset_cache(self):
@@ -91,8 +91,8 @@ class FileIngestionHistory(IngestionHistory):
 
     def _push_record(self, file_name, signature):
         self._history_dict[file_name] = signature
+
         self._history_file.write(f'{file_name},{signature}\n')
-        return None
 
     def _get_signature(self, file_name):
         return self._history_dict.get(file_name, None)
diff --git a/collection_manager/collection_manager/services/history_manager/IngestionHistory.py b/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
index b14b409..d92cb24 100644
--- a/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
@@ -53,6 +53,8 @@ class IngestionHistory(ABC):
         else:
             self._latest_ingested_file_update = max(self._latest_ingested_file_update, os.path.getmtime(file_path))
 
+        self._save_latest_timestamp()
+
     def latest_ingested_mtime(self) -> Optional[datetime]:
         """
         Return the modified time of the most recently modified file that was ingested.
@@ -100,6 +102,10 @@ class IngestionHistory(ABC):
             return GranuleStatus.UNDESIRED
 
     @abstractmethod
+    def _save_latest_timestamp(self):
+        pass
+
+    @abstractmethod
     def _push_record(self, file_name, signature):
         pass
 
diff --git a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
index 2d0438f..1ae7156 100644
--- a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
@@ -44,7 +44,6 @@ class SolrIngestionHistory(IngestionHistory):
             raise DatasetIngestionHistorySolrException(f"solr instance unreachable {solr_url}")
 
     def __del__(self):
-        self._push_latest_ingested_date()
         self._req_session.close()
 
     def _push_record(self, file_name, signature):
@@ -58,7 +57,7 @@ class SolrIngestionHistory(IngestionHistory):
         self._solr_granules.commit()
         return None
 
-    def _push_latest_ingested_date(self):
+    def _save_latest_timestamp(self):
         if self._solr_datasets:
             self._solr_datasets.delete(q=f"id:{self._dataset_id}")
             self._solr_datasets.add([{
diff --git a/collection_manager/tests/resources/collections_bad.yml b/collection_manager/tests/resources/collections_bad_schema.yml
similarity index 94%
copy from collection_manager/tests/resources/collections_bad.yml
copy to collection_manager/tests/resources/collections_bad_schema.yml
index cac6a32..37c6ad3 100644
--- a/collection_manager/tests/resources/collections_bad.yml
+++ b/collection_manager/tests/resources/collections_bad_schema.yml
@@ -1,10 +1,10 @@
-collections:
+bad_key:
   - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
     path: /opt/data/grace/*land*.nc
     variable: lwe_thickness
     priority: 1
     forward-processing-priority: 5
-BAD SYNTAX!
+
   - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_OCEAN
     path: /opt/data/grace/*ocean*.nc
     variable: lwe_thickness
diff --git a/collection_manager/tests/resources/collections_bad.yml b/collection_manager/tests/resources/collections_bad_syntax.yml
similarity index 100%
rename from collection_manager/tests/resources/collections_bad.yml
rename to collection_manager/tests/resources/collections_bad_syntax.yml
diff --git a/collection_manager/tests/services/test_CollectionProcessor.py b/collection_manager/tests/services/test_CollectionProcessor.py
index 7899e22..56d5393 100644
--- a/collection_manager/tests/services/test_CollectionProcessor.py
+++ b/collection_manager/tests/services/test_CollectionProcessor.py
@@ -46,7 +46,7 @@ class TestCollectionProcessor(unittest.TestCase):
 
         expected = """
         granule:
-          resource: test_path
+          resource: /granules/test_granule.nc
         processors:
           - name: GridReadingProcessor
             variable_to_read: test_variable
@@ -54,13 +54,13 @@ class TestCollectionProcessor(unittest.TestCase):
             dataset_name: test_dataset
             """
         collection = Collection(dataset_id="test_dataset",
-                                path="test_path",
+                                path="/granules/test*.nc",
                                 variable="test_variable",
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
                                 date_to=None)
-        filled = CollectionProcessor._fill_template(collection, template)
+        filled = CollectionProcessor._fill_template("/granules/test_granule.nc", collection, template)
         self.assertEqual(filled, expected)
 
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
diff --git a/collection_manager/tests/services/test_CollectionWatcher.py b/collection_manager/tests/services/test_CollectionWatcher.py
index 7ae25a1..8c6ab5f 100644
--- a/collection_manager/tests/services/test_CollectionWatcher.py
+++ b/collection_manager/tests/services/test_CollectionWatcher.py
@@ -6,7 +6,7 @@ from datetime import datetime
 from unittest.mock import Mock
 
 from collection_manager.entities import Collection
-from collection_manager.entities.exceptions import YamlParsingError, CollectionConfigFileNotFoundError, \
+from collection_manager.entities.exceptions import CollectionConfigParsingError, CollectionConfigFileNotFoundError, \
     RelativePathCollectionError, ConflictingPathCollectionError
 from collection_manager.services import CollectionWatcher
 
@@ -38,10 +38,16 @@ class TestCollectionWatcher(unittest.TestCase):
         self.assertEqual(len(collection_watcher._collections_by_dir['/opt/data/avhrr']), 1)
 
     def test_load_collections_with_bad_yaml_syntax(self):
-        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections_bad.yml')
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections_bad_syntax.yml')
         collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
 
-        self.assertRaises(YamlParsingError, collection_watcher._load_collections)
+        self.assertRaises(CollectionConfigParsingError, collection_watcher._load_collections)
+
+    def test_load_collections_with_bad_schema(self):
+        collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections_bad_schema.yml')
+        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+
+        self.assertRaises(CollectionConfigParsingError, collection_watcher._load_collections)
 
     def test_load_collections_with_file_not_found(self):
         collections_path = os.path.join(os.path.dirname(__file__), '../resources/does_not_exist.yml')

[incubator-sdap-ingester] 28/33: SDAP-302: Fix bug where the Collection Manager would crash if the Collections Config was updated while Collection Manager was actively scanning S3 directories (#29)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit d705545b03c971bc1592c404760f49126b57b0ab
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Fri Jan 8 15:19:19 2021 -0800

    SDAP-302: Fix bug where the Collection Manager would crash if the Collections Config was updated while Collection Manager was actively scanning S3 directories (#29)
---
 collection_manager/collection_manager/services/S3Observer.py | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/collection_manager/collection_manager/services/S3Observer.py b/collection_manager/collection_manager/services/S3Observer.py
index 6d87d91..f8efe4b 100644
--- a/collection_manager/collection_manager/services/S3Observer.py
+++ b/collection_manager/collection_manager/services/S3Observer.py
@@ -75,7 +75,11 @@ class S3Observer:
         new_cache = {}
         watch_index = {}
 
-        for watch in self._watches:
+        # We need to iterate on a copy of self._watches rather than on the original set itself
+        # because it is very possible that the original set could get updated while we are in the
+        # middle of scanning S3, which will cause an exception.
+        watches_copy = self._watches.copy()
+        for watch in watches_copy:
             new_cache_for_watch = await self._get_s3_files(watch.path)
             new_index = {file: watch for file in new_cache_for_watch}
 

[incubator-sdap-ingester] 04/33: SDAP-237 Dockerize Collection Manager (#4)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit cd4278526f192d1a9ea82561dcb1763c225cc090
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Thu Jun 25 16:10:27 2020 -0700

    SDAP-237 Dockerize Collection Manager (#4)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 collection_manager/README.md                       | 132 ++++++++-------------
 collection_manager/collection_manager/main.py      |  33 +++---
 collection_manager/containers/docker/Dockerfile    |  11 --
 .../containers/kubernetes/data-volume.yml          |  35 ------
 collection_manager/containers/kubernetes/job.yml   |  39 ------
 .../containers/kubernetes/sdap_ingester_config.yml |  38 ------
 collection_manager/docker/Dockerfile               |  16 +++
 collection_manager/docker/entrypoint.sh            |  10 ++
 collection_manager/setup.py                        |   8 +-
 granule_ingester/granule_ingester/main.py          |   3 +-
 10 files changed, 100 insertions(+), 225 deletions(-)

diff --git a/collection_manager/README.md b/collection_manager/README.md
index cbaf1fb..9d00cbb 100644
--- a/collection_manager/README.md
+++ b/collection_manager/README.md
@@ -1,103 +1,75 @@
-# SDAP manager for ingestion of datasets
+# SDAP Collection Manager
 
-## Prerequisites
-
-### python 3
-
-Install anaconda for python 3. From the graphic install for example for macos:
-
-https://www.anaconda.com/distribution/#macos
-
-### git lfs (for development)
-
-Git lfs for the deployment from git, see https://git-lfs.github.com/
-
-If not available you have to get netcdf files for test, if you do need the tests.
-
-### Deployed nexus on kubernetes cluster
+The SDAP Collection Manager is a service that watches a YAML file (the [Collections
+Configuration](#the-collections-configuration-file) file) stored on the filesystem, and all the directories listed in that
+file. Whenever new granules are added to any of the watched directories, the Collection
+Manager service will publish a message to RabbitMQ to be picked up by the Granule Ingester
+(`/granule_ingester` in this repository), which will then ingest the new granules.
 
-See project https://github.com/apache/incubator-sdap-nexus
 
-    $ helm install nexus .  --namespace=sdap --dependency-update -f ~/overridden-nexus-values.yml 
-
-For development purpose, you might want to expose solr port outside kubernetes
-
-   kubectl port-forward solr-set-0 8983:8983 -n sdap 
+## Prerequisites
 
- 
-## For developers
+Python 3.7
 
-### deploy project
+## Building the service
+From `incubator-sdap-ingester/collection_manager`, run:
 
-    $ bash
-    $ git clone ...
-    $ cd sdap_ingest_manager
-    $ python -m venv venv
-    $ source ./venv/bin/activate
-    $ pip install .
-    $ pytest -s
+    $ python setup.py install
     
-Note the command pip install -e . does not work as it does not deploy the configuration files.
-
-### Update the project
-
-Update the code and the test with your favorite IDE (e.g. pyCharm).
-
-### Launch for development/tests
-
-### Prerequisite
 
-Deploy a local rabbitmq service, for example with docker.
+## Running the service
+From `incubator-sdap-ingester/collection_manager`, run:
 
-    docker run -d --hostname localhost -p 5672:5672 --name rabbitmq rabbitmq:3
-   
-   
-### Launch the service
+    $ python collection_manager/main.py -h
+    
+### The Collections Configuration File
 
+A path to a collections configuration file must be passed in to the Collection Manager
+at startup via the `--collections-path` parameter. Below is an example of what the 
+collections configuration file should look like:
 
-The service reads the collection configuration and submit granule ingestion messages to the message broker (rabbitmq).
-For each collection, 2 ingestion priority levels are proposed: the nominal priority, the priority for forward processing (newer files), usually higher. 
-An history of the ingested granules is managed so that the ingestion can stop and re-start anytime.
+```yaml
+# collections.yaml
 
-    cd collection_manager
-    python main.py -h
-    python main.py --collections ../tests/resources/data/collections.yml --history-path=/tmp
+collections:
 
-# Containerization
+    # The identifier for the dataset as it will appear in NEXUS.
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND 
 
-TO BE UPDATED
+    # The local path to watch for NetCDF granule files to be associated with this dataset. 
+    # Supports glob-style patterns.
+    path: /opt/data/grace/*land*.nc 
 
-## Docker
+    # The name of the NetCDF variable to read when ingesting granules into NEXUS for this dataset.
+    variable: lwe_thickness 
 
-    docker build . -f containers/docker/config-operator/Dockerfile --no-cache --tag tloubrieu/sdap-ingest-manager:latest
-        
-To publish the docker image on dockerhub do (step necessary for kubernetes deployment):
+    # An integer priority level to use when publishing messages to RabbitMQ for historical data. 
+    # Higher number = higher priority.
+    priority: 1 
 
-    docker login
-    docker push tloubrieu/sdap-ingest-manager:latest
-    
-## Kubernetes
-    
-### Launch the service
+    # An integer priority level to use when publishing messages to RabbitMQ for forward-processing data.
+    # Higher number = higher priority.
+    forward-processing-priority: 5 
 
-    kubectl apply -f containers/kubernetes/job.yml -n sdap
-    
-Delete the service: 
+  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_OCEAN
+    path: /opt/data/grace/*ocean*.nc
+    variable: lwe_thickness
+    priority: 2
+    forward-processing-priority: 6
 
-    kubectl delete jobs --all -n sdap
-    
-    
+  - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
+    path: /opt/data/avhrr/*.nc
+    variable: analysed_sst
+    priority: 1
 
-    
+```
+## Running the tests
+From `incubator-sdap-ingester/collection_manager`, run:
 
+    $ pip install pytest
+    $ pytest
     
-    
-    
- 
-    
-    
-
-
-
-
+## Building the Docker image
+From `incubator-sdap-ingester/collection_manager`, run:
 
+    $ docker build . -f docker/Dockerfile -t nexusjpl/collection-manager
diff --git a/collection_manager/collection_manager/main.py b/collection_manager/collection_manager/main.py
index bc2d356..d8d2a5a 100644
--- a/collection_manager/collection_manager/main.py
+++ b/collection_manager/collection_manager/main.py
@@ -8,7 +8,7 @@ from collection_manager.services.history_manager import SolrIngestionHistoryBuil
 
 logging.basicConfig(level=logging.INFO)
 logging.getLogger("pika").setLevel(logging.WARNING)
-logger = logging.getLogger("collection_manager")
+logger = logging.getLogger(__name__)
 
 
 def check_path(path) -> str:
@@ -18,34 +18,35 @@ def check_path(path) -> str:
 
 
 def get_args() -> argparse.Namespace:
-    parser = argparse.ArgumentParser(description="Run ingestion for a list of collection ingestion streams")
-    parser.add_argument("--refresh",
-                        help="refresh interval in seconds to check for new or updated granules",
-                        default=300)
-    parser.add_argument("--collections",
+    parser = argparse.ArgumentParser(description="Watch the filesystem for new granules, and publish messages to "
+                                                 "RabbitMQ whenever they become available.")
+    parser.add_argument("--collections-path",
                         help="Absolute path to collections configuration file",
+                        metavar="PATH",
                         required=True)
-    parser.add_argument('--rabbitmq_host',
+    history_group = parser.add_mutually_exclusive_group(required=True)
+    history_group.add_argument("--history-path",
+                               metavar="PATH",
+                               help="Absolute path to ingestion history local directory")
+    history_group.add_argument("--history-url",
+                               metavar="URL",
+                               help="URL to ingestion history solr database")
+    parser.add_argument('--rabbitmq-host',
                         default='localhost',
                         metavar='HOST',
                         help='RabbitMQ hostname to connect to. (Default: "localhost")')
-    parser.add_argument('--rabbitmq_username',
+    parser.add_argument('--rabbitmq-username',
                         default='guest',
                         metavar='USERNAME',
                         help='RabbitMQ username. (Default: "guest")')
-    parser.add_argument('--rabbitmq_password',
+    parser.add_argument('--rabbitmq-password',
                         default='guest',
                         metavar='PASSWORD',
                         help='RabbitMQ password. (Default: "guest")')
-    parser.add_argument('--rabbitmq_queue',
+    parser.add_argument('--rabbitmq-queue',
                         default="nexus",
                         metavar="QUEUE",
                         help='Name of the RabbitMQ queue to consume from. (Default: "nexus")')
-    history_group = parser.add_mutually_exclusive_group(required=True)
-    history_group.add_argument("--history-path",
-                               help="Absolute path to ingestion history local directory")
-    history_group.add_argument("--history-url",
-                               help="URL to ingestion history solr database")
 
     return parser.parse_args()
 
@@ -65,7 +66,7 @@ def main():
         publisher.connect()
         collection_processor = CollectionProcessor(message_publisher=publisher,
                                                    history_manager_builder=history_manager_builder)
-        collection_watcher = CollectionWatcher(collections_path=options.collections,
+        collection_watcher = CollectionWatcher(collections_path=options.collections_path,
                                                collection_updated_callback=collection_processor.process_collection,
                                                granule_updated_callback=collection_processor.process_granule)
 
diff --git a/collection_manager/containers/docker/Dockerfile b/collection_manager/containers/docker/Dockerfile
deleted file mode 100644
index 3ba8da7..0000000
--- a/collection_manager/containers/docker/Dockerfile
+++ /dev/null
@@ -1,11 +0,0 @@
-FROM python:3
-
-# Add kubernetes client to create other pods (ingester)
-RUN apt-get update && apt-get install -y apt-transport-https gnupg2
-RUN curl -s https://packages.cloud.google.com/apt/doc/apt-key.gpg | apt-key add -
-RUN echo "deb https://apt.kubernetes.io/ kubernetes-xenial main" | tee -a /etc/apt/sources.list.d/kubernetes.list
-RUN apt-get update && apt-get install -y kubectl
-
-RUN pip install https://github.com/tloubrieu-jpl/incubator-sdap-nexus-ingestion-manager/releases/download/0.4.0+dev/sdap_ingest_manager-0.4.0+dev-py3-none-any.whl
-
-CMD bash
diff --git a/collection_manager/containers/kubernetes/data-volume.yml b/collection_manager/containers/kubernetes/data-volume.yml
deleted file mode 100644
index b2d3815..0000000
--- a/collection_manager/containers/kubernetes/data-volume.yml
+++ /dev/null
@@ -1,35 +0,0 @@
-apiVersion: v1
-kind: PersistentVolume
-metadata:
-  name: data-volume
-  labels:
-    name: data-volume
-spec:
-  capacity:
-    storage: 3Gi
-  volumeMode: Filesystem
-  accessModes:
-  - ReadWriteOnce
-  persistentVolumeReclaimPolicy: Delete
-  storageClassName: hostpath
-  hostPath:
-    path: /Users/loubrieu/PycharmProjects/sdap_ingest_manager/sdap_ingest_manager/ingestion_order_executor/history_manager/data
-    type: Directory
-
----
-
-apiVersion: v1
-kind: PersistentVolumeClaim
-metadata:
-  name: data-volume-claim
-spec:
-  accessModes:
-  - ReadWriteOnce
-  volumeMode: Filesystem
-  resources:
-    requests:
-      storage: 3Gi
-  storageClassName: hostpath
-  selector:
-    matchLabels:
-      name: "data-volume"
diff --git a/collection_manager/containers/kubernetes/job.yml b/collection_manager/containers/kubernetes/job.yml
deleted file mode 100644
index 1d8bc16..0000000
--- a/collection_manager/containers/kubernetes/job.yml
+++ /dev/null
@@ -1,39 +0,0 @@
-apiVersion: batch/v1
-kind: Job
-metadata:
-  name: collection-ingester
-spec:
-  template:
-    spec:
-      containers:
-        - name: collections-ingester
-          image: tloubrieu/sdap-ingest-manager:latest
-          imagePullPolicy: IfNotPresent
-          command: ["run_collections", "--config=/opt/sdap_ingester_config/"]
-          volumeMounts:
-            - name: config-vol
-              mountPath: /opt/sdap_ingester_config/
-            - name: data-volume-for-collection-ingester
-              mountPath: /data
-              readOnly: true
-      volumes:
-        - name: config-vol
-          configMap:
-            name: collection-ingester-config
-        - name: data-volume-for-collection-ingester
-          #hostPath:
-          #  path: /Users/loubrieu/PycharmProjects/sdap_ingest_manager/sdap_ingest_manager/ingestion_order_executor/history_manager/data
-          #  type: Directory
-          persistentVolumeClaim:
-            claimName: data-volume-claim
-
-      restartPolicy: Never
-  backoffLimit: 4
-
----
-
-
-
-
-
-
diff --git a/collection_manager/containers/kubernetes/sdap_ingester_config.yml b/collection_manager/containers/kubernetes/sdap_ingester_config.yml
deleted file mode 100644
index 425b687..0000000
--- a/collection_manager/containers/kubernetes/sdap_ingester_config.yml
+++ /dev/null
@@ -1,38 +0,0 @@
-apiVersion: v1
-data:
-  collections.yml: |+
-    # collection id with only letter and -
-    # path: regular expression matching the netcdf files which compose the collection
-    # variable: netcdf variable to be ingested (only one per dataset)
-    # priority: order in which collections will be processed, the smaller numbers first.
-    avhrr-oi-analysed-sst:
-      path: /data/avhrr_oi/*.nc
-      variable: analysed_sst
-      priority: 2
-
-  sdap_ingest_manager.ini: |+
-    [COLLECTIONS_YAML_CONFIG]
-    # config_path is the value sent as argument to the run_collection command, default is /opt/sdap_ingester_config
-    yaml_file = %(config_path)s/collections.yml
-
-    [OPTIONS]
-    # set to False to actually call the ingestion command for each granule
-    # relative path starts at {sys.prefix}/.sdap_ingest_manager
-    dry_run = False
-    # set to True to automatically list the granules as seen on the nfs server when they are mounted on the local file system.
-    deconstruct_nfs = False
-    # number of parallel ingestion pods on kubernetes (1 per granule)
-    parallel_pods = 8
-
-    [INGEST]
-    # kubernetes namespace where the sdap cluster is deployed
-    kubernetes_namespace = sdap
-
-
-kind: ConfigMap
-metadata:
-  creationTimestamp: "2020-04-17T00:11:46Z"
-  name: collection-ingester-config
-  resourceVersion: "2398917"
-  selfLink: /api/v1/namespaces/default/configmaps/collection-ingester
-  uid: b914e302-736c-4c25-9943-ebc33db418ce
diff --git a/collection_manager/docker/Dockerfile b/collection_manager/docker/Dockerfile
new file mode 100644
index 0000000..ce1b577
--- /dev/null
+++ b/collection_manager/docker/Dockerfile
@@ -0,0 +1,16 @@
+FROM python:3
+
+RUN apt-get update && apt-get install -y apt-transport-https gnupg2
+RUN curl -s https://packages.cloud.google.com/apt/doc/apt-key.gpg | apt-key add -
+RUN echo "deb https://apt.kubernetes.io/ kubernetes-xenial main" | tee -a /etc/apt/sources.list.d/kubernetes.list
+RUN apt-get update && apt-get install -y kubectl
+
+COPY /collection_manager /collection_manager/collection_manager
+COPY /setup.py /collection_manager/setup.py
+COPY /requirements.txt /collection_manager/requirements.txt
+COPY /README.md /collection_manager/README.md
+COPY /docker/entrypoint.sh /entrypoint.sh
+
+RUN cd /collection_manager && python setup.py install
+
+ENTRYPOINT ["/bin/bash", "/entrypoint.sh"]
diff --git a/collection_manager/docker/entrypoint.sh b/collection_manager/docker/entrypoint.sh
new file mode 100644
index 0000000..eb88f75
--- /dev/null
+++ b/collection_manager/docker/entrypoint.sh
@@ -0,0 +1,10 @@
+#!/bin/bash
+
+python /collection_manager/collection_manager/main.py \
+  $([[ ! -z "$COLLECTIONS_PATH" ]] && echo --collections-path=$COLLECTIONS_PATH) \
+  $([[ ! -z "$RABBITMQ_HOST" ]] && echo --rabbitmq-host=$RABBITMQ_HOST) \
+  $([[ ! -z "$RABBITMQ_USERNAME" ]] && echo --rabbitmq-username=$RABBITMQ_USERNAME) \
+  $([[ ! -z "$RABBITMQ_PASSWORD" ]] && echo --rabbitmq-password=$RABBITMQ_PASSWORD) \
+  $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq-queue=$RABBITMQ_QUEUE) \
+  $([[ ! -z "$HISTORY_URL" ]] && echo --history-url=$HISTORY_URL) \
+  $([[ ! -z "$HISTORY_PATH" ]] && echo --history-path=$HISTORY_PATH)
diff --git a/collection_manager/setup.py b/collection_manager/setup.py
index 49b0d75..1542486 100644
--- a/collection_manager/setup.py
+++ b/collection_manager/setup.py
@@ -1,9 +1,7 @@
-import setuptools
-import os
-import subprocess
-import sys
 import re
 
+import setuptools
+
 PACKAGE_NAME = "sdap_collection_manager"
 
 with open("./collection_manager/__init__.py") as fi:
@@ -24,7 +22,7 @@ setuptools.setup(
     description="a helper to ingest data in sdap",
     long_description=long_description,
     long_description_content_type="text/markdown",
-    url="https://github.com/tloubrieu-jpl/incubator-sdap-nexus-ingestion-manager",
+    url="https://github.com/apache/incubator-sdap-ingester",
     packages=setuptools.find_packages(),
     classifiers=[
         "Programming Language :: Python :: 3",
diff --git a/granule_ingester/granule_ingester/main.py b/granule_ingester/granule_ingester/main.py
index 29795f7..5a8fc2d 100644
--- a/granule_ingester/granule_ingester/main.py
+++ b/granule_ingester/granule_ingester/main.py
@@ -45,7 +45,8 @@ async def run_health_checks(dependencies: List[HealthCheck]):
 
 
 async def main():
-    parser = argparse.ArgumentParser(description='Process some integers.')
+    parser = argparse.ArgumentParser(description='Listen to RabbitMQ for granule ingestion instructions, and process '
+                                                 'and ingest a granule for each message that comes through.')
     parser.add_argument('--rabbitmq_host',
                         default='localhost',
                         metavar='HOST',

[incubator-sdap-ingester] 16/33: SDAP-279: Collection Manager should poll filesystem for file updates (#17)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 3fde4090264e6da81008ef283fe8963547b0dbd4
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Aug 11 10:38:08 2020 -0700

    SDAP-279: Collection Manager should poll filesystem for file updates (#17)
---
 .../collection_manager/services/CollectionWatcher.py         | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)

diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index 8911806..8f67e16 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -1,12 +1,12 @@
 import asyncio
+import time
 import logging
 import os
 from collections import defaultdict
 from typing import Dict, Callable, Set, Optional, Awaitable
 import yaml
 from watchdog.events import FileSystemEventHandler
-from watchdog.observers import Observer
-from yaml.scanner import ScannerError
+from watchdog.observers.polling import PollingObserver as Observer
 
 from collection_manager.entities import Collection
 from collection_manager.entities.exceptions import RelativePathError, CollectionConfigParsingError, \
@@ -102,9 +102,13 @@ class CollectionWatcher:
     async def _reload_and_reschedule(self):
         try:
             updated_collections = self._get_updated_collections()
-            for collection in updated_collections:
-                await self._collection_updated_callback(collection)
             if len(updated_collections) > 0:
+                logger.info(f"Scanning files for {len(updated_collections)} collections...")
+                start = time.perf_counter()
+                for collection in updated_collections:
+                    await self._collection_updated_callback(collection)
+                logger.info(f"Finished scanning files in {time.perf_counter() - start} seconds.")
+
                 self._unschedule_watches()
                 self._schedule_watches()
         except CollectionConfigParsingError as e:

[incubator-sdap-ingester] 29/33: add processor to make latitude ascending

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit e13d7ccd20f6721b2fd53b25acfa6db4d2a060c1
Author: thomas loubrieu <th...@jpl.nasa.gov>
AuthorDate: Wed Apr 7 22:36:43 2021 -0700

    add processor to make latitude ascending
---
 .../processors/ForceAscendingLatitude.py           | 48 ++++++++++++++++++
 .../processors/test_ForceAscendingLatitude.py      | 58 ++++++++++++++++++++++
 2 files changed, 106 insertions(+)

diff --git a/granule_ingester/granule_ingester/processors/ForceAscendingLatitude.py b/granule_ingester/granule_ingester/processors/ForceAscendingLatitude.py
new file mode 100644
index 0000000..01b2e88
--- /dev/null
+++ b/granule_ingester/granule_ingester/processors/ForceAscendingLatitude.py
@@ -0,0 +1,48 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You 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 numpy as np
+
+from nexusproto.serialization import from_shaped_array, to_shaped_array
+from granule_ingester.processors.TileProcessor import TileProcessor
+
+class ForceAscendingLatitude(TileProcessor):
+
+    def process(self, tile, *args, **kwargs):
+        """
+        This method will reverse the ordering of latitude values in a tile if necessary to ensure that the latitude values are ascending.
+​
+        :param self:
+        :param tile: The nexus_tile
+        :return: Tile data with altered latitude values
+        """
+
+        the_tile_type = tile.tile.WhichOneof("tile_type")
+
+        the_tile_data = getattr(tile.tile, the_tile_type)
+
+        latitudes = from_shaped_array(the_tile_data.latitude)
+
+        data = from_shaped_array(the_tile_data.variable_data)
+
+        # Only reverse latitude ordering if current ordering is descending.
+        if len(latitudes) > 1:
+            delta = latitudes[1] - latitudes[0]
+            if delta < 0:
+                latitudes = np.flip(latitudes)
+                data = np.flip(data, axis=0)
+                the_tile_data.latitude.CopyFrom(to_shaped_array(latitudes))
+                the_tile_data.variable_data.CopyFrom(to_shaped_array(data))
+
+        return tile
diff --git a/granule_ingester/tests/processors/test_ForceAscendingLatitude.py b/granule_ingester/tests/processors/test_ForceAscendingLatitude.py
new file mode 100644
index 0000000..d593909
--- /dev/null
+++ b/granule_ingester/tests/processors/test_ForceAscendingLatitude.py
@@ -0,0 +1,58 @@
+import unittest
+
+import xarray as xr
+import numpy as np
+from os import path
+from nexusproto import DataTile_pb2 as nexusproto
+
+from nexusproto.serialization import from_shaped_array, to_shaped_array
+
+from granule_ingester.processors.ForceAscendingLatitude import ForceAscendingLatitude
+from granule_ingester.processors.reading_processors.GridReadingProcessor import GridReadingProcessor
+
+class TestForceAscendingLatitude(unittest.TestCase):
+
+    def read_tile(self):
+        reading_processor = GridReadingProcessor('B03', 'lat', 'lon', time='time')
+        granule_path = path.join(path.dirname(__file__),
+                                 '/Users/loubrieu/Documents/sdap/HLS/HLS.S30.T11SPC.2020001.v1.4.hdf.nc')
+        input_tile = nexusproto.NexusTile()
+        input_tile.summary.granule = granule_path
+
+        dimensions_to_slices = {
+            'time': slice(0, 1),
+            'lat': slice(0, 30),
+            'lon': slice(0, 30)
+        }
+
+        with xr.open_dataset(granule_path) as ds:
+            return reading_processor._generate_tile(ds, dimensions_to_slices, input_tile)
+
+    def test_process(self):
+        processor = ForceAscendingLatitude()
+
+        tile = self.read_tile()
+
+        tile_type = tile.tile.WhichOneof("tile_type")
+        tile_data = getattr(tile.tile, tile_type)
+        latitudes = from_shaped_array(tile_data.latitude)
+        variable_data = from_shaped_array(tile_data.variable_data)
+        print(latitudes)
+        print(variable_data)
+
+
+        flipped_tile = processor.process(tile)
+
+        the_flipped_tile_type = flipped_tile.tile.WhichOneof("tile_type")
+        the_flipped_tile_data = getattr(flipped_tile.tile, the_flipped_tile_type)
+
+        flipped_latitudes = from_shaped_array(the_flipped_tile_data.latitude)
+        flipped_data = from_shaped_array(the_flipped_tile_data.variable_data)
+
+        print(flipped_latitudes[1])
+        np.testing.assert_almost_equal(flipped_latitudes[1], 38.72608, decimal=5, err_msg='', verbose=True)
+        print(flipped_data[1,1])
+        np.testing.assert_almost_equal(flipped_data[1,1], 0.3116, decimal=4, err_msg='', verbose=True)
+
+
+

[incubator-sdap-ingester] 18/33: SDAP-282, SDAP-284: Support configuring dimension names and projection during ingestion, support hierarchical directory structures (#19)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit e42ef8b143c7c4539ad24ee2441b557c46ca3b31
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Aug 25 10:47:56 2020 -0700

    SDAP-282, SDAP-284: Support configuring dimension names and projection during ingestion, support hierarchical directory structures (#19)
---
 .../collection_manager/entities/Collection.py      | 10 ++-
 .../resources/dataset_config_template.yml          | 20 ------
 .../services/CollectionProcessor.py                | 46 ++++++++-----
 .../services/CollectionWatcher.py                  | 15 +++--
 .../services/MessagePublisher.py                   |  2 +-
 collection_manager/setup.py                        |  1 -
 .../tests/entities/test_Collection.py              | 60 ++++++++++++++---
 collection_manager/tests/resources/collections.yml | 36 +++++++++-
 .../tests/resources/collections_alternate.yml      | 34 +++++++++-
 .../tests/services/test_CollectionProcessor.py     | 78 ++++++++++++++--------
 .../tests/services/test_CollectionWatcher.py       | 63 ++++++++++++++---
 config_operator/tests/resources/collections.yml    | 11 ++-
 .../consumer/{Consumer.py => MessageConsumer.py}   |  4 +-
 .../granule_ingester/consumer/__init__.py          |  2 +-
 .../granule_ingester/exceptions/Exceptions.py      |  7 +-
 .../granule_ingester/exceptions/__init__.py        | 19 +++---
 .../granule_loaders/GranuleLoader.py               |  9 ++-
 granule_ingester/granule_ingester/main.py          | 22 +++---
 .../granule_ingester/pipeline/Modules.py           | 17 +++--
 .../granule_ingester/pipeline/Pipeline.py          | 11 ++-
 .../reading_processors/EccoReadingProcessor.py     |  8 +--
 .../reading_processors/GridReadingProcessor.py     |  8 +--
 .../reading_processors/SwathReadingProcessor.py    |  6 +-
 .../reading_processors/TileReadingProcessor.py     | 10 +--
 .../TimeSeriesReadingProcessor.py                  |  8 +--
 .../config_files/ingestion_config_testfile.yaml    |  4 +-
 granule_ingester/tests/pipeline/test_Pipeline.py   |  7 +-
 .../test_EccoReadingProcessor.py                   |  4 +-
 .../test_GridReadingProcessor.py                   |  2 +-
 .../test_SwathReadingProcessor.py                  |  4 +-
 30 files changed, 358 insertions(+), 170 deletions(-)

diff --git a/collection_manager/collection_manager/entities/Collection.py b/collection_manager/collection_manager/entities/Collection.py
index 3976b6d..031a3a9 100644
--- a/collection_manager/collection_manager/entities/Collection.py
+++ b/collection_manager/collection_manager/entities/Collection.py
@@ -11,7 +11,9 @@ from collection_manager.entities.exceptions import MissingValueCollectionError
 @dataclass(frozen=True)
 class Collection:
     dataset_id: str
-    variable: str
+    projection: str
+    dimension_names: frozenset
+    slices: frozenset
     path: str
     historical_priority: int
     forward_processing_priority: Optional[int] = None
@@ -25,7 +27,9 @@ class Collection:
             date_from = datetime.fromisoformat(properties['from']) if 'from' in properties else None
 
             collection = Collection(dataset_id=properties['id'],
-                                    variable=properties['variable'],
+                                    projection=properties['projection'],
+                                    dimension_names=frozenset(properties['dimensionNames'].items()),
+                                    slices=frozenset(properties['slices'].items()),
                                     path=properties['path'],
                                     historical_priority=properties['priority'],
                                     forward_processing_priority=properties.get('forward-processing-priority', None),
@@ -51,4 +55,4 @@ class Collection:
             return fnmatch(file_path, self.path)
 
     def files_owned(self) -> List[str]:
-        return glob(self.path)
+        return glob(self.path, recursive=True)
diff --git a/collection_manager/collection_manager/resources/dataset_config_template.yml b/collection_manager/collection_manager/resources/dataset_config_template.yml
deleted file mode 100644
index d35a527..0000000
--- a/collection_manager/collection_manager/resources/dataset_config_template.yml
+++ /dev/null
@@ -1,20 +0,0 @@
-granule:
-  resource: {{granule}}
-slicer:
-  name: sliceFileByStepSize
-  dimension_step_sizes:
-    time: 1
-    lat: 30
-    lon: 30
-processors:
-  - name: GridReadingProcessor
-    latitude: lat
-    longitude: lon
-    time: time
-    variable_to_read: {{variable}}
-  - name: emptyTileFilter
-  - name: kelvinToCelsius
-  - name: tileSummary
-    dataset_name: {{dataset_id}}
-  - name: generateTileId
-
diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index ac61586..f08ade9 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -1,8 +1,7 @@
 import logging
 import os.path
 from typing import Dict
-
-import pystache
+import yaml
 
 from collection_manager.entities import Collection
 from collection_manager.services import MessagePublisher
@@ -11,8 +10,7 @@ from collection_manager.services.history_manager.IngestionHistory import Ingesti
 
 logger = logging.getLogger(__name__)
 
-SUPPORTED_FILE_EXTENSIONS = ['.nc', '.h5']
-MESSAGE_TEMPLATE = os.path.join(os.path.dirname(__file__), '../resources/dataset_config_template.yml')
+SUPPORTED_FILE_EXTENSIONS = ['.nc', '.nc4', '.h5']
 
 
 class CollectionProcessor:
@@ -22,9 +20,6 @@ class CollectionProcessor:
         self._history_manager_builder = history_manager_builder
         self._history_manager_cache: Dict[str, IngestionHistory] = {}
 
-        with open(MESSAGE_TEMPLATE, 'r') as config_template_file:
-            self._config_template = config_template_file.read()
-
     async def process_collection(self, collection: Collection):
         """
         Given a Collection, detect new granules that need to be ingested and publish RabbitMQ messages for each.
@@ -63,7 +58,7 @@ class CollectionProcessor:
                          f"collection '{collection.dataset_id}'. Skipping.")
             return
 
-        dataset_config = self._fill_template(granule, collection, config_template=self._config_template)
+        dataset_config = self._generate_ingestion_message(granule, collection)
         await self._publisher.publish_message(body=dataset_config, priority=use_priority)
         await history_manager.push(granule)
 
@@ -78,13 +73,28 @@ class CollectionProcessor:
         return self._history_manager_cache[dataset_id]
 
     @staticmethod
-    def _fill_template(granule_path: str, collection: Collection, config_template: str) -> str:
-        renderer = pystache.Renderer()
-        config_content = renderer.render(config_template,
-                                         {
-                                             'granule': granule_path,
-                                             'dataset_id': collection.dataset_id,
-                                             'variable': collection.variable
-                                         })
-        logger.debug(f"Templated dataset config:\n{config_content}")
-        return config_content
+    def _generate_ingestion_message(granule_path: str, collection: Collection) -> str:
+        config_dict = {
+            'granule': {
+                'resource': granule_path
+            },
+            'slicer': {
+                'name': 'sliceFileByStepSize',
+                'dimension_step_sizes': dict(collection.slices)
+            },
+            'processors': [
+                {
+                    'name': collection.projection,
+                    **dict(collection.dimension_names),
+                },
+                {'name': 'emptyTileFilter'},
+                {
+                    'name': 'tileSummary',
+                    'dataset_name': collection.dataset_id
+                },
+                {'name': 'generateTileId'}
+            ]
+        }
+        config_str = yaml.dump(config_dict)
+        logger.debug(f"Templated dataset config:\n{config_str}")
+        return config_str
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index 54c8877..1c7c1be 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -50,7 +50,7 @@ class CollectionWatcher:
                                      func=self._reload_and_reschedule)
         self._observer.start()
 
-    def collections(self) -> Set[Collection]:
+    def _collections(self) -> Set[Collection]:
         """
         Return a set of all Collections being watched.
         :return: A set of Collections
@@ -96,9 +96,9 @@ class CollectionWatcher:
                                                "next file modification.")
 
     def _get_updated_collections(self) -> Set[Collection]:
-        old_collections = self.collections()
+        old_collections = self._collections()
         self._load_collections()
-        return self.collections() - old_collections
+        return self._collections() - old_collections
 
     async def _reload_and_reschedule(self):
         try:
@@ -128,7 +128,7 @@ class CollectionWatcher:
             # Note: the Watchdog library does not schedule a new watch
             # if one is already scheduled for the same directory
             try:
-                self._granule_watches.add(self._observer.schedule(granule_event_handler, directory))
+                self._granule_watches.add(self._observer.schedule(granule_event_handler, directory, recursive=True))
             except (FileNotFoundError, NotADirectoryError):
                 bad_collection_names = ' and '.join([col.dataset_id for col in collections])
                 logger.error(f"Granule directory {directory} does not exist. Ignoring {bad_collection_names}.")
@@ -169,8 +169,11 @@ class _GranuleEventHandler(FileSystemEventHandler):
     def on_created(self, event):
         super().on_created(event)
         for collection in self._collections_for_dir:
-            if collection.owns_file(event.src_path):
-                self._loop.create_task(self._callback(event.src_path, collection))
+            try:
+                if collection.owns_file(event.src_path):
+                    self._loop.create_task(self._callback(event.src_path, collection))
+            except IsADirectoryError:
+                pass
 
     def on_modified(self, event):
         super().on_modified(event)
diff --git a/collection_manager/collection_manager/services/MessagePublisher.py b/collection_manager/collection_manager/services/MessagePublisher.py
index 75803d1..483fff8 100644
--- a/collection_manager/collection_manager/services/MessagePublisher.py
+++ b/collection_manager/collection_manager/services/MessagePublisher.py
@@ -25,7 +25,7 @@ class MessagePublisher:
         """
         self._connection = await connect_robust(self._connection_string)
         self._channel = await self._connection.channel()
-        await self._channel.declare_queue(self._queue, durable=True)
+        await self._channel.declare_queue(self._queue, durable=True, arguments={'x-max-priority': 10})
 
     @retry(wait=wait_fixed(5), reraise=True, stop=stop_after_attempt(4))
     async def publish_message(self, body: str, priority: int = None):
diff --git a/collection_manager/setup.py b/collection_manager/setup.py
index 1542486..0616d0f 100644
--- a/collection_manager/setup.py
+++ b/collection_manager/setup.py
@@ -31,6 +31,5 @@ setuptools.setup(
     ],
     python_requires='>=3.6',
     include_package_data=True,
-    data_files=[('.collection_manager/resources/', ['collection_manager/resources/dataset_config_template.yml'])],
     install_requires=pip_requirements
 )
diff --git a/collection_manager/tests/entities/test_Collection.py b/collection_manager/tests/entities/test_Collection.py
index 46506d4..7e56c9d 100644
--- a/collection_manager/tests/entities/test_Collection.py
+++ b/collection_manager/tests/entities/test_Collection.py
@@ -12,7 +12,9 @@ class TestCollection(unittest.TestCase):
         directory = os.path.join(os.path.dirname(__file__), "../resources/data")
         collection = Collection(dataset_id="test_dataset",
                                 path=directory,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -23,7 +25,9 @@ class TestCollection(unittest.TestCase):
         pattern = os.path.join(os.path.dirname(__file__), "../resources/data/*.nc")
         collection = Collection(dataset_id="test_dataset",
                                 path=pattern,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -34,7 +38,9 @@ class TestCollection(unittest.TestCase):
         directory = os.path.join(os.path.dirname(__file__), "../resources/data")
         collection = Collection(dataset_id="test_dataset",
                                 path=directory,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -45,7 +51,9 @@ class TestCollection(unittest.TestCase):
         directory = os.path.join(os.path.dirname(__file__), "../resources/data")
         collection = Collection(dataset_id="test_dataset",
                                 path=directory,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -57,7 +65,9 @@ class TestCollection(unittest.TestCase):
         directory = os.path.join(os.path.dirname(__file__), "../resources/data")
         collection = Collection(dataset_id="test_dataset",
                                 path=directory,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -69,7 +79,9 @@ class TestCollection(unittest.TestCase):
         pattern = os.path.join(directory, "test_*.nc")
         collection = Collection(dataset_id="test_dataset",
                                 path=pattern,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -82,7 +94,9 @@ class TestCollection(unittest.TestCase):
         pattern = os.path.join(directory, "test_*.nc")
         collection = Collection(dataset_id="test_dataset",
                                 path=pattern,
-                                variable="test_variable",
+                                projection="Grid",
+                                slices={},
+                                dimension_names={},
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -93,8 +107,14 @@ class TestCollection(unittest.TestCase):
     def test_from_dict(self):
         collection_dict = {
             'id': 'test_id',
-            'variable': 'test_var',
             'path': '/some/path',
+            'projection': 'Grid',
+            'dimensionNames': {
+                'latitude': 'lat',
+                'longitude': 'lon',
+                'variable': 'test_var'
+            },
+            'slices': {'lat': 30, 'lon': 30, 'time': 1},
             'priority': 1,
             'forward-processing-priority': 2,
             'from': '2020-01-01T00:00:00+00:00',
@@ -102,7 +122,13 @@ class TestCollection(unittest.TestCase):
         }
 
         expected_collection = Collection(dataset_id='test_id',
-                                         variable='test_var',
+                                         projection="Grid",
+                                         slices=frozenset([('lat', 30), ('lon', 30), ('time', 1)]),
+                                         dimension_names=frozenset([
+                                             ('latitude', 'lat'),
+                                             ('longitude', 'lon'),
+                                             ('variable', 'test_var')
+                                         ]),
                                          path='/some/path',
                                          historical_priority=1,
                                          forward_processing_priority=2,
@@ -114,13 +140,25 @@ class TestCollection(unittest.TestCase):
     def test_from_dict_missing_optional_values(self):
         collection_dict = {
             'id': 'test_id',
-            'variable': 'test_var',
+            'projection': 'Grid',
+            'dimensionNames': {
+                'latitude': 'lat',
+                'longitude': 'lon',
+                'variable': 'test_var'
+            },
+            'slices': {'lat': 30, 'lon': 30, 'time': 1},
             'path': '/some/path',
             'priority': 3
         }
 
         expected_collection = Collection(dataset_id='test_id',
-                                         variable='test_var',
+                                         projection="Grid",
+                                         slices=frozenset([('lat', 30), ('lon', 30), ('time', 1)]),
+                                         dimension_names=frozenset([
+                                             ('latitude', 'lat'),
+                                             ('longitude', 'lon'),
+                                             ('variable', 'test_var')
+                                         ]),
                                          path='/some/path',
                                          historical_priority=3,
                                          forward_processing_priority=None,
diff --git a/collection_manager/tests/resources/collections.yml b/collection_manager/tests/resources/collections.yml
index 89524ec..44f795b 100644
--- a/collection_manager/tests/resources/collections.yml
+++ b/collection_manager/tests/resources/collections.yml
@@ -1,17 +1,47 @@
 collections:
   - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
     path: /opt/data/grace/*land*.nc
-    variable: lwe_thickness
     priority: 1
     forward-processing-priority: 5
+    projection: Grid
+    dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+    slices:
+      time: 1
+      lat: 30
+      lon: 30
+
 
   - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_OCEAN
     path: /opt/data/grace/*ocean*.nc
-    variable: lwe_thickness
     priority: 2
     forward-processing-priority: 6
+    projection: Grid
+    dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+    slices:
+      time: 1
+      lat: 30
+      lon: 30
+
 
   - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
     path: /opt/data/avhrr/*.nc
-    variable: analysed_sst
     priority: 1
+    projection: Grid
+    dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: analysed_sst
+    slices:
+      time: 1
+      lat: 30
+      lon: 30
+
diff --git a/collection_manager/tests/resources/collections_alternate.yml b/collection_manager/tests/resources/collections_alternate.yml
index 3d7da95..f9dabda 100644
--- a/collection_manager/tests/resources/collections_alternate.yml
+++ b/collection_manager/tests/resources/collections_alternate.yml
@@ -1,17 +1,45 @@
 collections:
   - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
     path: /opt/data/grace/*land*.nc
-    variable: lwe_thickness
     priority: 1
     forward-processing-priority: 5
+    projection: Grid
+    dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+    slices:
+      time: 1
+      lat: 30
+      lon: 30
 
   - id: ID_CHANGED
     path: /opt/data/grace/*ocean*.nc
-    variable: lwe_thickness
     priority: 2
     forward-processing-priority: 6
+    projection: Grid
+    dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+    slices:
+      time: 1
+      lat: 30
+      lon: 30
 
   - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
     path: /opt/data/avhrr/*.nc
-    variable: analysed_sst
     priority: 1
+    projection: Grid
+    dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: analysed_sst
+    slices:
+      time: 1
+      lat: 30
+      lon: 30
+
diff --git a/collection_manager/tests/services/test_CollectionProcessor.py b/collection_manager/tests/services/test_CollectionProcessor.py
index a7059d6..939b5d1 100644
--- a/collection_manager/tests/services/test_CollectionProcessor.py
+++ b/collection_manager/tests/services/test_CollectionProcessor.py
@@ -1,4 +1,5 @@
 import tempfile
+import yaml
 import unittest
 from unittest import mock
 
@@ -35,34 +36,47 @@ class TestCollectionProcessor(unittest.TestCase):
             self.assertIsNot(collection_processor._get_history_manager('bar'), history_manager)
 
     def test_fill_template(self):
-        template = """
-        granule:
-          resource: {{granule}}
-        processors:
-          - name: GridReadingProcessor
-            variable_to_read: {{variable}}
-          - name: tileSummary
-            dataset_name: {{dataset_id}}
-            """
-
-        expected = """
-        granule:
-          resource: /granules/test_granule.nc
-        processors:
-          - name: GridReadingProcessor
-            variable_to_read: test_variable
-          - name: tileSummary
-            dataset_name: test_dataset
-            """
+        expected = {
+            'granule': {
+                'resource': '/granules/test_granule.nc'
+            },
+            'processors': [
+                {
+                    'latitude': 'lat',
+                    'longitude': 'lon',
+                    'name': 'Grid',
+                    'variable': 'test_var'
+                },
+                {'name': 'emptyTileFilter'},
+                {'dataset_name': 'test_dataset', 'name': 'tileSummary'},
+                {'name': 'generateTileId'}
+            ],
+            'slicer': {
+                'dimension_step_sizes': {
+                    'lat': 30,
+                    'lon': 30,
+                    'time': 1
+                },
+                'name': 'sliceFileByStepSize'
+            }
+        }
         collection = Collection(dataset_id="test_dataset",
                                 path="/granules/test*.nc",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset([('lat', 30), ('lon', 30), ('time', 1)]),
+                                dimension_names=frozenset([
+                                    ('latitude', 'lat'),
+                                    ('longitude', 'lon'),
+                                    ('variable', 'test_var')
+                                ]),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
                                 date_to=None)
-        filled = CollectionProcessor._fill_template("/granules/test_granule.nc", collection, template)
-        self.assertEqual(filled, expected)
+        filled = CollectionProcessor._generate_ingestion_message("/granules/test_granule.nc", collection)
+        generated_yaml = yaml.load(filled, Loader=yaml.FullLoader)
+
+        self.assertEqual(expected, generated_yaml)
 
     @async_test
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', new_callable=AsyncMock)
@@ -75,7 +89,9 @@ class TestCollectionProcessor(unittest.TestCase):
         collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
         collection = Collection(dataset_id="test_dataset",
                                 path="test_path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -100,7 +116,9 @@ class TestCollectionProcessor(unittest.TestCase):
         collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
         collection = Collection(dataset_id="test_dataset",
                                 path="test_path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -123,7 +141,9 @@ class TestCollectionProcessor(unittest.TestCase):
         collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
         collection = Collection(dataset_id="test_dataset",
                                 path="test_path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 date_from=None,
                                 date_to=None)
@@ -144,7 +164,9 @@ class TestCollectionProcessor(unittest.TestCase):
         collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
         collection = Collection(dataset_id="test_dataset",
                                 path="test_path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -165,7 +187,9 @@ class TestCollectionProcessor(unittest.TestCase):
         collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
         collection = Collection(dataset_id="test_dataset",
                                 path="test_path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
diff --git a/collection_manager/tests/services/test_CollectionWatcher.py b/collection_manager/tests/services/test_CollectionWatcher.py
index c9a75c0..e6bf15f 100644
--- a/collection_manager/tests/services/test_CollectionWatcher.py
+++ b/collection_manager/tests/services/test_CollectionWatcher.py
@@ -25,7 +25,7 @@ class TestCollectionWatcher(unittest.TestCase):
                 Collection("id4", "var4", "path4", 7, 8, datetime.now(), datetime.now()),
             }
         }
-        flattened_collections = collection_watcher.collections()
+        flattened_collections = collection_watcher._collections()
         self.assertEqual(len(flattened_collections), 4)
 
     def test_load_collections_loads_all_collections(self):
@@ -60,7 +60,7 @@ class TestCollectionWatcher(unittest.TestCase):
         collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
 
         updated_collections = collection_watcher._get_updated_collections()
-        self.assertSetEqual(updated_collections, collection_watcher.collections())
+        self.assertSetEqual(updated_collections, collection_watcher._collections())
 
     def test_get_updated_collections_returns_no_collections(self):
         collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
@@ -87,7 +87,9 @@ class TestCollectionWatcher(unittest.TestCase):
 
         collection = Collection(dataset_id="test_dataset",
                                 path="/absolute/path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -100,7 +102,9 @@ class TestCollectionWatcher(unittest.TestCase):
 
         collection = Collection(dataset_id="test_dataset",
                                 path="relative/path",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -113,7 +117,9 @@ class TestCollectionWatcher(unittest.TestCase):
 
         collection = Collection(dataset_id="test_dataset",
                                 path="/resources/*.nc",
-                                variable="test_variable",
+                                projection="Grid",
+                                slices=frozenset(),
+                                dimension_names=frozenset(),
                                 historical_priority=1,
                                 forward_processing_priority=2,
                                 date_from=None,
@@ -127,9 +133,19 @@ class TestCollectionWatcher(unittest.TestCase):
         collections_str = f"""collections:
 - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
   path: {granule_dir.name}
-  variable: lwe_thickness
   priority: 1
-  forward-processing-priority: 5"""
+  forward-processing-priority: 5
+  projection: Grid
+  dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+  slices:
+      time: 1
+      lat: 30
+      lon: 30
+  """
         collections_config.write(collections_str.encode("utf-8"))
 
         collection_callback = AsyncMock()
@@ -143,9 +159,18 @@ class TestCollectionWatcher(unittest.TestCase):
         collections_str = f"""
 - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
   path: {granule_dir.name}
-  variable: lwe_thickness
   priority: 10
   forward-processing-priority: 5
+  projection: Grid
+  dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+  slices:
+      time: 1
+      lat: 30
+      lon: 30
         """
         collections_config.write(collections_str.encode("utf-8"))
 
@@ -163,9 +188,18 @@ class TestCollectionWatcher(unittest.TestCase):
 collections:
 - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
   path: {granule_dir.name}
-  variable: lwe_thickness
   priority: 1
   forward-processing-priority: 5
+  projection: Grid
+  dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+  slices:
+      time: 1
+      lat: 30
+      lon: 30
             """
             collections_config.write(collections_str.encode("utf-8"))
 
@@ -187,9 +221,18 @@ collections:
 collections:
 - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
   path: {granule_dir.name}
-  variable: lwe_thickness
   priority: 1
   forward-processing-priority: 5
+  projection: Grid
+  dimensionNames:
+      latitude: lat
+      longitude: lon
+      time: time
+      variable: lwe_thickness
+  slices:
+      time: 1
+      lat: 30
+      lon: 30
             """
             collections_config.write(collections_str.encode("utf-8"))
             new_granule = open(os.path.join(granule_dir.name, 'test.nc'), "w+")
diff --git a/config_operator/tests/resources/collections.yml b/config_operator/tests/resources/collections.yml
index 42d2fbc..3414c5a 100644
--- a/config_operator/tests/resources/collections.yml
+++ b/config_operator/tests/resources/collections.yml
@@ -5,5 +5,14 @@ avhrr-oi-analysed-sst:
 
 avhrr-oi-analysed-sst2:
   path: resources/history_manager/data/avhrr_oi/*.nc
-  variable: analysed_sst
   priority: 1
+  projection: Grid
+  dimensionNames:
+    latitude: lat
+    longitude: lon
+    time: time
+    variable: analysed_sst
+  slices:
+    time: 1
+    lat: 30
+    lon: 30
diff --git a/granule_ingester/granule_ingester/consumer/Consumer.py b/granule_ingester/granule_ingester/consumer/MessageConsumer.py
similarity index 98%
rename from granule_ingester/granule_ingester/consumer/Consumer.py
rename to granule_ingester/granule_ingester/consumer/MessageConsumer.py
index 6c72837..4d6c07b 100644
--- a/granule_ingester/granule_ingester/consumer/Consumer.py
+++ b/granule_ingester/granule_ingester/consumer/MessageConsumer.py
@@ -25,7 +25,7 @@ from granule_ingester.pipeline import Pipeline
 logger = logging.getLogger(__name__)
 
 
-class Consumer(HealthCheck):
+class MessageConsumer(HealthCheck):
 
     def __init__(self,
                  rabbitmq_host,
@@ -95,7 +95,7 @@ class Consumer(HealthCheck):
     async def start_consuming(self, pipeline_max_concurrency=16):
         channel = await self._connection.channel()
         await channel.set_qos(prefetch_count=1)
-        queue = await channel.declare_queue(self._rabbitmq_queue, durable=True)
+        queue = await channel.declare_queue(self._rabbitmq_queue, durable=True, arguments={'x-max-priority': 10})
         queue_iter = queue.iterator()
         async for message in queue_iter:
             try:
diff --git a/granule_ingester/granule_ingester/consumer/__init__.py b/granule_ingester/granule_ingester/consumer/__init__.py
index 35d075b..bb782d5 100644
--- a/granule_ingester/granule_ingester/consumer/__init__.py
+++ b/granule_ingester/granule_ingester/consumer/__init__.py
@@ -1 +1 @@
-from granule_ingester.consumer.Consumer import Consumer
+from granule_ingester.consumer.MessageConsumer import MessageConsumer
diff --git a/granule_ingester/granule_ingester/exceptions/Exceptions.py b/granule_ingester/granule_ingester/exceptions/Exceptions.py
index c648b99..fdd03e5 100644
--- a/granule_ingester/granule_ingester/exceptions/Exceptions.py
+++ b/granule_ingester/granule_ingester/exceptions/Exceptions.py
@@ -6,7 +6,11 @@ class PipelineRunningError(Exception):
     pass
 
 
-class TileProcessingError(Exception):
+class TileProcessingError(PipelineRunningError):
+    pass
+
+
+class GranuleLoadingError(PipelineRunningError):
     pass
 
 
@@ -21,6 +25,7 @@ class RabbitMQLostConnectionError(LostConnectionError):
 class CassandraLostConnectionError(LostConnectionError):
     pass
 
+
 class SolrLostConnectionError(LostConnectionError):
     pass
 
diff --git a/granule_ingester/granule_ingester/exceptions/__init__.py b/granule_ingester/granule_ingester/exceptions/__init__.py
index ea0969f..f2429b1 100644
--- a/granule_ingester/granule_ingester/exceptions/__init__.py
+++ b/granule_ingester/granule_ingester/exceptions/__init__.py
@@ -1,11 +1,8 @@
-from .Exceptions import CassandraFailedHealthCheckError
-from .Exceptions import CassandraLostConnectionError
-from .Exceptions import FailedHealthCheckError
-from .Exceptions import LostConnectionError
-from .Exceptions import PipelineBuildingError
-from .Exceptions import PipelineRunningError
-from .Exceptions import RabbitMQFailedHealthCheckError
-from .Exceptions import RabbitMQLostConnectionError
-from .Exceptions import SolrFailedHealthCheckError
-from .Exceptions import SolrLostConnectionError
-from .Exceptions import TileProcessingError
+from .Exceptions import (CassandraFailedHealthCheckError,
+                         CassandraLostConnectionError, FailedHealthCheckError,
+                         GranuleLoadingError, LostConnectionError,
+                         PipelineBuildingError, PipelineRunningError,
+                         RabbitMQFailedHealthCheckError,
+                         RabbitMQLostConnectionError,
+                         SolrFailedHealthCheckError, SolrLostConnectionError,
+                         TileProcessingError)
diff --git a/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py b/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py
index c28ffbb..6377de0 100644
--- a/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py
+++ b/granule_ingester/granule_ingester/granule_loaders/GranuleLoader.py
@@ -21,6 +21,8 @@ from urllib import parse
 import aioboto3
 import xarray as xr
 
+from granule_ingester.exceptions import GranuleLoadingError
+
 logger = logging.getLogger(__name__)
 
 
@@ -52,7 +54,12 @@ class GranuleLoader:
             raise RuntimeError("Granule path scheme '{}' is not supported.".format(resource_url.scheme))
 
         granule_name = os.path.basename(self._resource)
-        return xr.open_dataset(file_path, lock=False), granule_name
+        try:
+            return xr.open_dataset(file_path, lock=False), granule_name
+        except FileNotFoundError:
+            raise GranuleLoadingError(f"The granule file {self._resource} does not exist.")
+        except Exception:
+            raise GranuleLoadingError(f"The granule {self._resource} is not a valid NetCDF file.")
 
     @staticmethod
     async def _download_s3_file(url: str):
diff --git a/granule_ingester/granule_ingester/main.py b/granule_ingester/granule_ingester/main.py
index 15390fd..b5a429c 100644
--- a/granule_ingester/granule_ingester/main.py
+++ b/granule_ingester/granule_ingester/main.py
@@ -20,7 +20,7 @@ import sys
 from functools import partial
 from typing import List
 
-from granule_ingester.consumer import Consumer
+from granule_ingester.consumer import MessageConsumer
 from granule_ingester.exceptions import FailedHealthCheckError, LostConnectionError
 from granule_ingester.healthcheck import HealthCheck
 from granule_ingester.writers import CassandraStore, SolrStore
@@ -116,16 +116,16 @@ async def main(loop):
     solr_host_and_port = args.solr_host_and_port
     zk_host_and_port = args.zk_host_and_port
 
-    consumer = Consumer(rabbitmq_host=args.rabbitmq_host,
-                        rabbitmq_username=args.rabbitmq_username,
-                        rabbitmq_password=args.rabbitmq_password,
-                        rabbitmq_queue=args.rabbitmq_queue,
-                        data_store_factory=partial(cassandra_factory,
-                                                   cassandra_contact_points,
-                                                   cassandra_port,
-                                                   cassandra_username,
-                                                   cassandra_password),
-                        metadata_store_factory=partial(solr_factory, solr_host_and_port, zk_host_and_port))
+    consumer = MessageConsumer(rabbitmq_host=args.rabbitmq_host,
+                               rabbitmq_username=args.rabbitmq_username,
+                               rabbitmq_password=args.rabbitmq_password,
+                               rabbitmq_queue=args.rabbitmq_queue,
+                               data_store_factory=partial(cassandra_factory,
+                                                          cassandra_contact_points,
+                                                          cassandra_port,
+                                                          cassandra_username,
+                                                          cassandra_password),
+                               metadata_store_factory=partial(solr_factory, solr_host_and_port, zk_host_and_port))
     try:
         solr_store = SolrStore(zk_url=zk_host_and_port) if zk_host_and_port else SolrStore(solr_url=solr_host_and_port)
         await run_health_checks([CassandraStore(cassandra_contact_points,
diff --git a/granule_ingester/granule_ingester/pipeline/Modules.py b/granule_ingester/granule_ingester/pipeline/Modules.py
index 2cf2245..d1950dc 100644
--- a/granule_ingester/granule_ingester/pipeline/Modules.py
+++ b/granule_ingester/granule_ingester/pipeline/Modules.py
@@ -1,14 +1,19 @@
-from granule_ingester.processors import *
-from granule_ingester.processors.reading_processors import *
-from granule_ingester.slicers import *
-from granule_ingester.granule_loaders import *
+from granule_ingester.processors import GenerateTileId, TileSummarizingProcessor, EmptyTileFilter, KelvinToCelsius
+from granule_ingester.processors.reading_processors import (EccoReadingProcessor,
+                                                            GridReadingProcessor,
+                                                            SwathReadingProcessor,
+                                                            TimeSeriesReadingProcessor)
+from granule_ingester.slicers import SliceFileByStepSize
+from granule_ingester.granule_loaders import GranuleLoader
 
 modules = {
     "granule": GranuleLoader,
     "sliceFileByStepSize": SliceFileByStepSize,
     "generateTileId": GenerateTileId,
-    "EccoReadingProcessor": EccoReadingProcessor,
-    "GridReadingProcessor": GridReadingProcessor,
+    "ECCO": EccoReadingProcessor,
+    "Grid": GridReadingProcessor,
+    "TimeSeries": TimeSeriesReadingProcessor,
+    "Swath": SwathReadingProcessor,
     "tileSummary": TileSummarizingProcessor,
     "emptyTileFilter": EmptyTileFilter,
     "kelvinToCelsius": KelvinToCelsius
diff --git a/granule_ingester/granule_ingester/pipeline/Pipeline.py b/granule_ingester/granule_ingester/pipeline/Pipeline.py
index dabca81..86bf9c8 100644
--- a/granule_ingester/granule_ingester/pipeline/Pipeline.py
+++ b/granule_ingester/granule_ingester/pipeline/Pipeline.py
@@ -153,10 +153,13 @@ class Pipeline:
                        metadata_store_factory,
                        tile_processors,
                        max_concurrency)
+        except PipelineBuildingError:
+            raise
         except KeyError as e:
             raise PipelineBuildingError(f"Cannot build pipeline because {e} is missing from the YAML.")
-        except Exception:
-            raise PipelineBuildingError("Cannot build pipeline.")
+        except Exception as e:
+            logger.exception(e)
+            raise PipelineBuildingError(f"Cannot build pipeline because of the following error: {e}")
 
     @classmethod
     def _parse_module(cls, module_config: dict, module_mappings: dict):
@@ -166,7 +169,9 @@ class Pipeline:
             logger.debug("Loaded processor {}.".format(module_class))
             processor_module = module_class(**module_config)
         except KeyError:
-            raise RuntimeError("'{}' is not a valid processor.".format(module_name))
+            raise PipelineBuildingError(f"'{module_name}' is not a valid processor.")
+        except Exception as e:
+            raise PipelineBuildingError(f"Parsing module '{module_name}' failed because of the following error: {e}")
 
         return processor_module
 
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py
index 1876013..8cc24d0 100644
--- a/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py
+++ b/granule_ingester/granule_ingester/processors/reading_processors/EccoReadingProcessor.py
@@ -10,14 +10,14 @@ from granule_ingester.processors.reading_processors.TileReadingProcessor import
 
 class EccoReadingProcessor(TileReadingProcessor):
     def __init__(self,
-                 variable_to_read,
+                 variable,
                  latitude,
                  longitude,
                  tile,
                  depth=None,
                  time=None,
                  **kwargs):
-        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+        super().__init__(variable, latitude, longitude, **kwargs)
 
         self.depth = depth
         self.time = time
@@ -31,8 +31,8 @@ class EccoReadingProcessor(TileReadingProcessor):
         lat_subset = np.ma.filled(np.squeeze(lat_subset), np.NaN)
         lon_subset = np.ma.filled(np.squeeze(lon_subset), np.NaN)
 
-        data_subset = ds[self.variable_to_read][
-            type(self)._slices_for_variable(ds[self.variable_to_read], dimensions_to_slices)]
+        data_subset = ds[self.variable][
+            type(self)._slices_for_variable(ds[self.variable], dimensions_to_slices)]
         data_subset = np.ma.filled(np.squeeze(data_subset), np.NaN)
 
         new_tile.tile = ds[self.tile][dimensions_to_slices[self.tile].start].item()
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py
index 4354f9e..1ba76a2 100644
--- a/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py
+++ b/granule_ingester/granule_ingester/processors/reading_processors/GridReadingProcessor.py
@@ -9,8 +9,8 @@ from granule_ingester.processors.reading_processors.TileReadingProcessor import
 
 
 class GridReadingProcessor(TileReadingProcessor):
-    def __init__(self, variable_to_read, latitude, longitude, depth=None, time=None, **kwargs):
-        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+    def __init__(self, variable, latitude, longitude, depth=None, time=None, **kwargs):
+        super().__init__(variable, latitude, longitude, **kwargs)
         self.depth = depth
         self.time = time
 
@@ -22,8 +22,8 @@ class GridReadingProcessor(TileReadingProcessor):
         lat_subset = np.ma.filled(np.squeeze(lat_subset), np.NaN)
         lon_subset = np.ma.filled(np.squeeze(lon_subset), np.NaN)
 
-        data_subset = ds[self.variable_to_read][type(self)._slices_for_variable(ds[self.variable_to_read],
-                                                                                dimensions_to_slices)]
+        data_subset = ds[self.variable][type(self)._slices_for_variable(ds[self.variable],
+                                                                        dimensions_to_slices)]
         data_subset = np.ma.filled(np.squeeze(data_subset), np.NaN)
 
         if self.depth:
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py
index fec28ca..5b8072a 100644
--- a/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py
+++ b/granule_ingester/granule_ingester/processors/reading_processors/SwathReadingProcessor.py
@@ -9,8 +9,8 @@ from granule_ingester.processors.reading_processors.TileReadingProcessor import
 
 
 class SwathReadingProcessor(TileReadingProcessor):
-    def __init__(self, variable_to_read, latitude, longitude, time, depth=None, **kwargs):
-        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+    def __init__(self, variable, latitude, longitude, time, depth=None, **kwargs):
+        super().__init__(variable, latitude, longitude, **kwargs)
         self.depth = depth
         self.time = time
 
@@ -25,7 +25,7 @@ class SwathReadingProcessor(TileReadingProcessor):
         time_subset = ds[self.time][type(self)._slices_for_variable(ds[self.time], dimensions_to_slices)]
         time_subset = np.ma.filled(type(self)._convert_to_timestamp(time_subset), np.NaN)
 
-        data_subset = ds[self.variable_to_read][type(self)._slices_for_variable(ds[self.variable_to_read],
+        data_subset = ds[self.variable][type(self)._slices_for_variable(ds[self.variable],
                                                                                 dimensions_to_slices)]
         data_subset = np.ma.filled(data_subset, np.NaN)
 
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
index 8b69ad2..aa70db3 100644
--- a/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
+++ b/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
@@ -27,8 +27,8 @@ from granule_ingester.processors.TileProcessor import TileProcessor
 
 class TileReadingProcessor(TileProcessor, ABC):
 
-    def __init__(self, variable_to_read: str, latitude: str, longitude: str, *args, **kwargs):
-        self.variable_to_read = variable_to_read
+    def __init__(self, variable: str, latitude: str, longitude: str, *args, **kwargs):
+        self.variable = variable
         self.latitude = latitude
         self.longitude = longitude
 
@@ -38,11 +38,11 @@ class TileReadingProcessor(TileProcessor, ABC):
 
             output_tile = nexusproto.NexusTile()
             output_tile.CopyFrom(tile)
-            output_tile.summary.data_var_name = self.variable_to_read
+            output_tile.summary.data_var_name = self.variable
 
             return self._generate_tile(dataset, dimensions_to_slices, output_tile)
-        except Exception:
-            raise TileProcessingError("Could not generate tiles from the granule.")
+        except Exception as e:
+            raise TileProcessingError(f"Could not generate tiles from the granule because of the following error: {e}.")
 
     @abstractmethod
     def _generate_tile(self, dataset: xr.Dataset, dimensions_to_slices: Dict[str, slice], tile):
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py
index 2831c0c..c4aae25 100644
--- a/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py
+++ b/granule_ingester/granule_ingester/processors/reading_processors/TimeSeriesReadingProcessor.py
@@ -9,8 +9,8 @@ from granule_ingester.processors.reading_processors.TileReadingProcessor import
 
 
 class TimeSeriesReadingProcessor(TileReadingProcessor):
-    def __init__(self, variable_to_read, latitude, longitude, time, depth=None, **kwargs):
-        super().__init__(variable_to_read, latitude, longitude, **kwargs)
+    def __init__(self, variable, latitude, longitude, time, depth=None, **kwargs):
+        super().__init__(variable, latitude, longitude, **kwargs)
 
         self.depth = depth
         self.time = time
@@ -23,8 +23,8 @@ class TimeSeriesReadingProcessor(TileReadingProcessor):
         lat_subset = np.ma.filled(lat_subset, np.NaN)
         lon_subset = np.ma.filled(lon_subset, np.NaN)
 
-        data_subset = ds[self.variable_to_read][type(self)._slices_for_variable(ds[self.variable_to_read],
-                                                                                dimensions_to_slices)]
+        data_subset = ds[self.variable][type(self)._slices_for_variable(ds[self.variable],
+                                                                        dimensions_to_slices)]
         data_subset = np.ma.filled(data_subset, np.NaN)
 
         if self.depth:
diff --git a/granule_ingester/tests/config_files/ingestion_config_testfile.yaml b/granule_ingester/tests/config_files/ingestion_config_testfile.yaml
index 9af889d..63df51a 100644
--- a/granule_ingester/tests/config_files/ingestion_config_testfile.yaml
+++ b/granule_ingester/tests/config_files/ingestion_config_testfile.yaml
@@ -7,11 +7,11 @@ slicer:
     lat: 33
     lon: 26
 processors:
-  - name: EccoReadingProcessor
+  - name: ECCO
     latitude: YC
     longitude: XC
     time: time
     depth: Z
     tile: tile
-    variable_to_read: THETA
+    variable: THETA
   - name: generateTileId
diff --git a/granule_ingester/tests/pipeline/test_Pipeline.py b/granule_ingester/tests/pipeline/test_Pipeline.py
index 34e66c6..27ec72b 100644
--- a/granule_ingester/tests/pipeline/test_Pipeline.py
+++ b/granule_ingester/tests/pipeline/test_Pipeline.py
@@ -6,8 +6,9 @@ from nexusproto import DataTile_pb2 as nexusproto
 from granule_ingester.pipeline.Pipeline import Pipeline
 from granule_ingester.processors import GenerateTileId
 from granule_ingester.processors.reading_processors import EccoReadingProcessor
-from granule_ingester.slicers.SliceFileByStepSize import *
+from granule_ingester.slicers.SliceFileByStepSize import SliceFileByStepSize
 from granule_ingester.writers import DataStore, MetadataStore
+from granule_ingester.exceptions import PipelineBuildingError
 
 
 class TestPipeline(unittest.TestCase):
@@ -70,7 +71,7 @@ class TestPipeline(unittest.TestCase):
             "name": "MockModule",
             "bogus_param": True
         }
-        self.assertRaises(TypeError, Pipeline._parse_module, module_config, module_mappings)
+        self.assertRaises(PipelineBuildingError, Pipeline._parse_module, module_config, module_mappings)
 
     def test_parse_module_with_missing_parameters(self):
         module_mappings = {"MockModule": TestPipeline.MockProcessorWithParams}
@@ -78,7 +79,7 @@ class TestPipeline(unittest.TestCase):
             "name": "MockModule"
         }
 
-        self.assertRaises(TypeError, Pipeline._parse_module, module_config, module_mappings)
+        self.assertRaises(PipelineBuildingError, Pipeline._parse_module, module_config, module_mappings)
 
     def test_process_tile(self):
         # class MockIdProcessor:
diff --git a/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py b/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py
index f2e9f29..03d5054 100644
--- a/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py
+++ b/granule_ingester/tests/reading_processors/test_EccoReadingProcessor.py
@@ -10,7 +10,7 @@ from granule_ingester.processors.reading_processors import EccoReadingProcessor
 class TestEccoReadingProcessor(unittest.TestCase):
 
     def test_generate_tile(self):
-        reading_processor = EccoReadingProcessor(variable_to_read='OBP',
+        reading_processor = EccoReadingProcessor(variable='OBP',
                                                  latitude='YC',
                                                  longitude='XC',
                                                  time='time',
@@ -40,7 +40,7 @@ class TestEccoReadingProcessor(unittest.TestCase):
             self.assertEqual(output_tile.tile.ecco_tile.longitude.shape, [15, 7])
 
     def test_generate_tile_with_dims_out_of_order(self):
-        reading_processor = EccoReadingProcessor(variable_to_read='OBP',
+        reading_processor = EccoReadingProcessor(variable='OBP',
                                                  latitude='YC',
                                                  longitude='XC',
                                                  time='time',
diff --git a/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py b/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py
index aec3ae8..31cb547 100644
--- a/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py
+++ b/granule_ingester/tests/reading_processors/test_GridReadingProcessor.py
@@ -209,7 +209,7 @@ class TestReadInterpEccoData(unittest.TestCase):
                                            time='time')
 
     def test_read_indexed_ecco(self):
-        reading_processor = GridReadingProcessor(variable_to_read='OBP',
+        reading_processor = GridReadingProcessor(variable='OBP',
                                                  latitude='latitude',
                                                  longitude='longitude',
                                                  time='time')
diff --git a/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py b/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py
index 55ac4fc..db623f5 100644
--- a/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py
+++ b/granule_ingester/tests/reading_processors/test_SwathReadingProcessor.py
@@ -24,7 +24,7 @@ from granule_ingester.processors.reading_processors import SwathReadingProcessor
 
 class TestReadAscatbData(unittest.TestCase):
     def test_read_not_empty_ascatb(self):
-        reading_processor = SwathReadingProcessor(variable_to_read='wind_speed',
+        reading_processor = SwathReadingProcessor(variable='wind_speed',
                                                   latitude='lat',
                                                   longitude='lon',
                                                   time='time')
@@ -50,7 +50,7 @@ class TestReadAscatbData(unittest.TestCase):
 class TestReadSmapData(unittest.TestCase):
     def test_read_not_empty_smap(self):
         reading_processor = SwathReadingProcessor(
-            variable_to_read='smap_sss',
+            variable='smap_sss',
             latitude='lat',
             longitude='lon',
             time='row_time')

[incubator-sdap-ingester] 33/33: temporarilly remove too big test granule

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit e101a17ac6d8c5941239641456ad077474defcea
Author: thomas loubrieu <th...@jpl.nasa.gov>
AuthorDate: Thu Apr 8 17:19:34 2021 -0700

    temporarilly remove too big test granule

[incubator-sdap-ingester] 31/33: replace hls test data with smaller one

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit aea467dddad34feab813c17b0d8f50a5ceb16870
Author: thomas loubrieu <th...@jpl.nasa.gov>
AuthorDate: Thu Apr 8 10:48:03 2021 -0700

    replace hls test data with smaller one

[incubator-sdap-ingester] 09/33: Fix syntax error (#9)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 4f7fdf2e61a5c6b89d87122910a35253eb3d14f5
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Mon Aug 3 19:26:25 2020 -0700

    Fix syntax error (#9)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 .../collection_manager/services/history_manager/SolrIngestionHistory.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
index 319fa3e..79d6eef 100644
--- a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
@@ -105,7 +105,7 @@ class SolrIngestionHistory(IngestionHistory):
                 logger.info(f"solr collection created {response}")
 
                 # Update schema
-                schema_endpoint = f"{self._url_prefix/{self._granule_collection_name}/schema"
+                schema_endpoint = f"{self._url_prefix}/{self._granule_collection_name}/schema"
                 self._add_field(schema_endpoint, "dataset_s", "string")
                 self._add_field(schema_endpoint, "granule_s", "string")
                 self._add_field(schema_endpoint, "granule_signature_s", "string")

[incubator-sdap-ingester] 10/33: add a on.update kopf event for crd updates

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit cf30aad9e4c4ae0ce8caa3ce6d0c4858d445386e
Author: thomas loubrieu <th...@jpl.nasa.gov>
AuthorDate: Tue Aug 4 13:17:47 2020 -0700

    add a on.update kopf event for crd updates
    
    organize logs in operator
---
 config_operator/README.md                          | 10 +++++-
 .../config_source/RemoteGitConfig.py               |  4 +--
 config_operator/config_operator/main.py            | 38 +++++++++++++++++-----
 .../containers/k8s/config-operator-crd.yml         |  2 +-
 .../containers/k8s/deployment-git-src.yml          | 23 -------------
 config_operator/containers/k8s/git-repo-test.yml   |  4 +--
 6 files changed, 44 insertions(+), 37 deletions(-)

diff --git a/config_operator/README.md b/config_operator/README.md
index ba4c6fc..5f02804 100644
--- a/config_operator/README.md
+++ b/config_operator/README.md
@@ -28,7 +28,11 @@ To publish the docker image on dockerhub do (step necessary for kubernetes deplo
     docker push nexusjpl/config-operator:latest
     
 ## Kubernetes
-    
+
+Delete pre-existing operator definitions:
+
+    kubectl delete deployment.apps/git-repo-config-operator  -n sdap
+   
 Deploy the gitbasedconfig operator:
 
      kubectl apply -f containers/k8s/config-operator-crd.yml -n sdap
@@ -45,6 +49,10 @@ Check that the configMap has been generated:
 
     kubectl get configmaps -n sdap
     
+Test an update of the config operator configuration:
+
+    kubectl set image gitbasedconfig/nginx-deployment nginx=nginx:1.16.1 --record
+    
 
     
     
\ No newline at end of file
diff --git a/config_operator/config_operator/config_source/RemoteGitConfig.py b/config_operator/config_operator/config_source/RemoteGitConfig.py
index 38cbe2d..350fcfc 100644
--- a/config_operator/config_operator/config_source/RemoteGitConfig.py
+++ b/config_operator/config_operator/config_source/RemoteGitConfig.py
@@ -39,8 +39,8 @@ class RemoteGitConfig(LocalDirConfig):
         self._git_token = git_token
         if local_dir is None:
             local_dir = DEFAULT_LOCAL_REPO_DIR
-        self._update_every_seconds = update_every_seconds
-        super().__init__(local_dir, update_every_seconds=self._update_every_seconds)
+        super().__init__(local_dir, update_every_seconds=update_every_seconds)
+
 
         if repo:
             self._repo = repo
diff --git a/config_operator/config_operator/main.py b/config_operator/config_operator/main.py
index fac6741..45d530f 100644
--- a/config_operator/config_operator/main.py
+++ b/config_operator/config_operator/main.py
@@ -8,11 +8,7 @@ logging.basicConfig(level=logging.INFO)
 logger = logging.getLogger(__name__)
 
 
-@kopf.on.create('sdap.apache.org', 'v1', 'gitbasedconfigs')
-def create_fn(body, spec, **kwargs):
-    # Get info from Git Repo Config object
-    namespace = body['metadata']['namespace']
-
+def create_config_synchronizer(spec, namespace):
     if 'git-url' not in spec.keys():
         raise kopf.HandlerFatalError(f"git-url must be set.")
     if 'config-map' not in spec.keys():
@@ -23,22 +19,48 @@ def create_fn(body, spec, **kwargs):
     config_map = spec['config-map']
     logger.info(f'config-map = {config_map}')
 
-    _kargs = {}
+    _kwargs = {}
     for k in {'git-branch', 'git-username', 'git-token', 'update-every-seconds'}:
         if k in spec:
             logger.info(f'{k} = {spec[k]}')
-            _kargs[k.replace('-', '_')] = spec[k]
+            _kwargs[k.replace('-', '_')] = spec[k]
 
-    config = RemoteGitConfig(git_url, **_kargs)
+    config = RemoteGitConfig(git_url, **_kwargs)
 
     config_map = K8sConfigMap(config_map, namespace, config)
 
     asyncio.run(config.when_updated(config_map.publish))
 
     msg = f"configmap {config_map} created from git repo {git_url}"
+
+    return msg
+
+
+@kopf.on.create('sdap.apache.org', 'v1', 'gitbasedconfigs')
+def create_fn(body, spec, **kwargs):
+    logger.info(f'sdap config operator creation')
+
+    namespace = body['metadata']['namespace']
+
+    msg = create_config_synchronizer(spec, namespace)
+
+    logger.info(f'sdap config operator created {msg}')
+
+    return {'message': msg}
+
+
+@kopf.on.update('sdap.apache.org', 'v1', 'gitbasedconfigs')
+def update_fn(spec, status, namespace, **kwargs):
+    logger.info(f'sdap config operator update')
+
+    msg = create_config_synchronizer(spec, namespace)
+
+    logger.info(f'sdap config operator updated {msg}')
+
     return {'message': msg}
 
 
+
 @kopf.on.login()
 def login_fn(**kwargs):
     return kopf.login_via_client(**kwargs)
diff --git a/config_operator/containers/k8s/config-operator-crd.yml b/config_operator/containers/k8s/config-operator-crd.yml
index 9f6d2ad..2159c00 100644
--- a/config_operator/containers/k8s/config-operator-crd.yml
+++ b/config_operator/containers/k8s/config-operator-crd.yml
@@ -12,7 +12,7 @@ spec:
   names:
     plural: gitbasedconfigs
     singular: gitbasedconfig
-    kind: gitBasedConfig
+    kind: GitBasedConfig
     shortNames:
     - gitcfg
   validation:
diff --git a/config_operator/containers/k8s/deployment-git-src.yml b/config_operator/containers/k8s/deployment-git-src.yml
deleted file mode 100644
index 99b268d..0000000
--- a/config_operator/containers/k8s/deployment-git-src.yml
+++ /dev/null
@@ -1,23 +0,0 @@
-apiVersion: apps/v1
-kind: Deployment
-metadata:
-  name: config-operator
-  labels:
-    app: sdap-config-operator
-spec:
-  replicas: 1
-  selector:
-    matchLabels:
-      app: sdap-config-operator
-  template:
-    metadata:
-      labels:
-        app: sdap-config-operator
-    spec:
-      containers:
-      - name: sdap-config-operator
-        image: tloubrieu/config-operator:latest
-        imagePullPolicy: IfNotPresent
-        command: ['config-operator', '--git-url', 'https://github.com/tloubrieu-jpl/sdap-ingester-config' , '--namespace', 'sdap', '--config-map', 'collection-ingester-conf', '-u']
-
-
diff --git a/config_operator/containers/k8s/git-repo-test.yml b/config_operator/containers/k8s/git-repo-test.yml
index 6a98454..79ad518 100644
--- a/config_operator/containers/k8s/git-repo-test.yml
+++ b/config_operator/containers/k8s/git-repo-test.yml
@@ -1,9 +1,9 @@
 apiVersion: sdap.apache.org/v1
-kind: gitBasedConfig
+kind: GitBasedConfig
 metadata:
   name: collections-config-gitcfg
 spec:
   git-url: https://github.com/tloubrieu-jpl/sdap-ingester-config.git
-  git-branch: master
+  git-branch: bigdata
   git-token: whatever
   config-map: my-configmap
\ No newline at end of file

[incubator-sdap-ingester] 23/33: SDAP-293: Automatic Kelvin-to-Celsius conversion (#22)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 0d87acb96a23c71e8d42e9b127e003d7f1f3d90b
Author: Elizabeth Yam <ec...@users.noreply.github.com>
AuthorDate: Wed Nov 18 14:11:58 2020 -0800

    SDAP-293: Automatic Kelvin-to-Celsius conversion (#22)
    
    Co-authored-by: Eamon Ford <ea...@gmail.com>
---
 .../services/CollectionProcessor.py                  |  1 +
 granule_ingester/granule_ingester/README.md          | 11 +++++++++++
 .../granule_ingester/processors/TileProcessor.py     | 20 +++++++++++++++++++-
 .../granule_ingester/processors/kelvintocelsius.py   | 18 +++++++++++-------
 4 files changed, 42 insertions(+), 8 deletions(-)

diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index 96c461e..bab56fc 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -87,6 +87,7 @@ class CollectionProcessor:
                 },
                 {'name': 'emptyTileFilter'},
                 {'name': 'subtract180FromLongitude'},
+                {'name': 'kelvinToCelsius'},
                 {
                     'name': 'tileSummary',
                     'dataset_name': collection.dataset_id
diff --git a/granule_ingester/granule_ingester/README.md b/granule_ingester/granule_ingester/README.md
new file mode 100644
index 0000000..881461a
--- /dev/null
+++ b/granule_ingester/granule_ingester/README.md
@@ -0,0 +1,11 @@
+## Plugin Processor Architecture
+The operator can write a class that inherits from the `TileProcessor` class and implements the abstract function `process`, which among other things, takes in the NexusTile(`nexusproto.DataTile_pb2.NexusTile` object)) generated with default configurations and the NC4 Dataset(`xarray.Dataset` object), and allows the user to add further modifications to how granule data is saved.  
+
+Any additional transformation the operator needs to accomplish must be done in this `process` method, which is what is ultimately called in the ingestion pipeline.  Helper functions are suggested for breaking up complex procedures.
+
+The custom code file would be copied into the ingestion pods via the helm chart (see chart for local and mount paths).
+
+Example: `KelvinToCelsiusProcessor`
+This processor checks the units of the saved variable.  If it is some form of Kelvin, it automatically converts all of the temperature measurements to Celsius by subtracting 273.15 from each data point.  The transformed data then replaces the default (untransformed) values and the processor returns the modified tile.
+
+#### TODO Add configuration option for unusual representations of temperature units.
\ No newline at end of file
diff --git a/granule_ingester/granule_ingester/processors/TileProcessor.py b/granule_ingester/granule_ingester/processors/TileProcessor.py
index d62c504..47b1a2f 100644
--- a/granule_ingester/granule_ingester/processors/TileProcessor.py
+++ b/granule_ingester/granule_ingester/processors/TileProcessor.py
@@ -14,10 +14,28 @@
 # limitations under the License.
 
 from abc import ABC, abstractmethod
+from nexusproto.serialization import from_shaped_array, to_shaped_array
+from nexusproto.DataTile_pb2 import NexusTile
 
 
 # TODO: make this an informal interface, not an abstract class
 class TileProcessor(ABC):
     @abstractmethod
-    def process(self, tile, *args, **kwargs):
+    def process(self, tile: NexusTile, *args, **kwargs):
+        # accessing the data
+        # the_tile_type = tile.tile.WhichOneof("tile_type")
+        # the_tile_data = getattr(tile.tile, the_tile_type)
+
+        # get netCDF as xarray.Dataset object
+        # ds = kwargs['dataset']
+
+        # example transformation:
+        # var_data = from_shaped_array(the_tile_data.variable_data) - 273.15
+
+        # save transformed data back into tile
+        # the_tile_data.variable_data.CopyFrom(to_shaped_array(var_data))
+
+        # return transformed tile
+        # return tile
+
         pass
diff --git a/granule_ingester/granule_ingester/processors/kelvintocelsius.py b/granule_ingester/granule_ingester/processors/kelvintocelsius.py
index e728418..9ad4f49 100644
--- a/granule_ingester/granule_ingester/processors/kelvintocelsius.py
+++ b/granule_ingester/granule_ingester/processors/kelvintocelsius.py
@@ -13,19 +13,23 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-
 from nexusproto.serialization import from_shaped_array, to_shaped_array
-
+from nexusproto.DataTile_pb2 import NexusTile
 from granule_ingester.processors.TileProcessor import TileProcessor
 
 
 class KelvinToCelsius(TileProcessor):
-    def process(self, tile, *args, **kwargs):
+    def process(self, tile: NexusTile, *args, **kwargs):
         the_tile_type = tile.tile.WhichOneof("tile_type")
         the_tile_data = getattr(tile.tile, the_tile_type)
-
-        var_data = from_shaped_array(the_tile_data.variable_data) - 273.15
-
-        the_tile_data.variable_data.CopyFrom(to_shaped_array(var_data))
+        kelvins = ['kelvin', 'degk', 'deg_k', 'degreesk', 'degrees_k', 'degree_k', 'degreek']
+
+        if 'dataset' in kwargs:
+            ds = kwargs['dataset']
+            variable_name = tile.summary.data_var_name
+            variable_unit = ds.variables[variable_name].attrs['units']
+            if any([unit in variable_unit.lower() for unit in kelvins]):
+                var_data = from_shaped_array(the_tile_data.variable_data) - 273.15
+                the_tile_data.variable_data.CopyFrom(to_shaped_array(var_data))
 
         return tile

[incubator-sdap-ingester] 25/33: SDAP-297: Update Collections Config docs to match latest schema (#26)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 528ba21d64d61607f358afee587b0f1b6e2843c3
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Mon Dec 7 14:22:15 2020 -0800

    SDAP-297: Update Collections Config docs to match latest schema (#26)
---
 collection_manager/README.md | 81 ++++++++++++++++++++++++++++++++------------
 1 file changed, 60 insertions(+), 21 deletions(-)

diff --git a/collection_manager/README.md b/collection_manager/README.md
index 84df468..90e72fa 100644
--- a/collection_manager/README.md
+++ b/collection_manager/README.md
@@ -26,7 +26,7 @@ From `incubator-sdap-ingester`, run:
 
 A path to a collections configuration file must be passed in to the Collection Manager
 at startup via the `--collections-path` parameter. Below is an example of what the 
-collections configuration file should look like:
+collections configuration file could look like:
 
 ```yaml
 # collections.yaml
@@ -34,35 +34,74 @@ collections configuration file should look like:
 collections:
 
     # The identifier for the dataset as it will appear in NEXUS.
-  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND 
+  - id: "CSR-RL06-Mascons_LAND"
 
-    # The local path to watch for NetCDF granule files to be associated with this dataset. 
-    # Supports glob-style patterns.
-    path: /opt/data/grace/*land*.nc 
-
-    # The name of the NetCDF variable to read when ingesting granules into NEXUS for this dataset.
-    variable: lwe_thickness 
+    # The path to watch for NetCDF granule files to be associated with this dataset. 
+    # This can also be an S3 path prefix, for example "s3://my-bucket/path/to/granules/"
+    path: "/data/CSR-RL06-Mascons-land/" 
 
     # An integer priority level to use when publishing messages to RabbitMQ for historical data. 
-    # Higher number = higher priority.
-    priority: 1 
+    # Higher number = higher priority. Scale is 1-10.
+    priority: 1
 
     # An integer priority level to use when publishing messages to RabbitMQ for forward-processing data.
-    # Higher number = higher priority.
+    # Higher number = higher priority. Scale is 1-10.
     forward-processing-priority: 5 
 
-  - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_OCEAN
-    path: /opt/data/grace/*ocean*.nc
-    variable: lwe_thickness
-    priority: 2
-    forward-processing-priority: 6
+    # The type of project to use when processing granules in this collection.
+    # Accepted values are Grid, ECCO, TimeSeries, or Swath.
+    projection: Grid
+
+    dimensionNames:
+      # The name of the primary variable
+      variable: lwe_thickness
+
+      # The name of the latitude variable
+      latitude: lat
+
+      # The name of the longitude variable
+      longitude: lon
+
+      # The name of the depth variable (only include if depth variable exists)
+      depth: Z 
+      
+      # The name of the time variable (only include if time variable exists)
+      time: Time
+
+    # This section is an index of each dimension on which the primary variable is dependent, mapped to their desired slice sizes.
+    slices:
+      Z: 1 
+      Time: 1
+      lat: 60
+      lon: 60
+
+ - id: ocean-bottom-pressure 
+    path: /data/OBP/
+    priority: 6
+    forward-processing-priority: 7
+    projection: ECCO
+    dimensionNames:
+      latitude: YC
+      longitude: XC
+      time: time
+      # "tile" is required when using the ECCO projection. This refers to the name of the dimension containing the ECCO tile index.
+      tile: tile
+      variable: OBP
+    slices:
+      time: 1
+      tile: 1
+      i: 30
+      j: 30
+```
+
+Note that the dimensions listed under `slices` will not necessarily match the values of the properties under `dimensionNames`. This is because sometimes
+the actual dimensions are referenced by index variables. 
 
-  - id: AVHRR_OI-NCEI-L4-GLOB-v2.0
-    path: /opt/data/avhrr/*.nc
-    variable: analysed_sst
-    priority: 1
+> **Tip:** An easy way to determine which variables go under `dimensionNames` and which ones go under `slices` is that the variables 
+> on which the primary variable is dependent should be listed under `slices`, and the variables on which _those_ variables are dependent 
+> (which could be themselves, as in the case of the first collection in the above example) should be the values of the properties under 
+> `dimensionNames`. The excepction to this is that `dimensionNames.variable` should always be the name of the primary variable.
 
-```
 ## Running the tests
 From `incubator-sdap-ingester/`, run:
 

[incubator-sdap-ingester] 06/33: SDAP-247: config-operator unit tests and support for git username/token (#6)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 5256cddee70efa2ac2869ce4229b4792f20ab42c
Author: thomas loubrieu <60...@users.noreply.github.com>
AuthorDate: Wed Jul 29 17:54:25 2020 -0700

    SDAP-247: config-operator unit tests and support for git username/token  (#6)
    
    Co-authored-by: thomas loubrieu <th...@jpl.nasa.gov>
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 config_operator/README.md                          |  6 +-
 .../config_source/LocalDirConfig.py                | 40 ++++++------
 .../config_source/RemoteGitConfig.py               | 51 +++++++++------
 .../config_operator/k8s/K8sConfigMap.py            | 26 +++++---
 config_operator/config_operator/main.py            | 12 +++-
 config_operator/containers/k8s/git-repo-test.yml   |  9 +++
 config_operator/requirements.txt                   |  1 +
 .../tests/config_source/test_LocalDirConfig.py     | 72 ++++++++++++++++++++++
 .../tests/config_source/test_RemoteGitConfig.py    | 49 +++++++++++++++
 config_operator/tests/k8s/test_K8sConfigMap.py     | 53 +++++++++++++++-
 .../resources/localDirBadTest/collections.yml      |  2 +
 .../tests/resources/localDirTest/.hidden_file.txt  |  1 +
 .../tests/resources/localDirTest/README.md         |  1 +
 .../tests/resources/localDirTest/collections.yml   |  1 +
 14 files changed, 269 insertions(+), 55 deletions(-)

diff --git a/config_operator/README.md b/config_operator/README.md
index 4624a0f..ba4c6fc 100644
--- a/config_operator/README.md
+++ b/config_operator/README.md
@@ -16,16 +16,16 @@ The component runs as a kubernetes operator (see containerization section)
     pip install -e .
     pytest -d
 
-# Containerizaion
+# Containerization
 
 ## Docker
 
-    docker build . -f containers/docker/Dockerfile --no-cache --tag tloubrieu/config-operator:latest
+    docker build . -f containers/docker/Dockerfile -t nexusjpl/config-operator:latest
         
 To publish the docker image on dockerhub do (step necessary for kubernetes deployment):
 
     docker login
-    docker push tloubrieu/sdap-ingest-manager:latest
+    docker push nexusjpl/config-operator:latest
     
 ## Kubernetes
     
diff --git a/config_operator/config_operator/config_source/LocalDirConfig.py b/config_operator/config_operator/config_source/LocalDirConfig.py
index cf95f42..555a7fa 100644
--- a/config_operator/config_operator/config_source/LocalDirConfig.py
+++ b/config_operator/config_operator/config_source/LocalDirConfig.py
@@ -1,10 +1,11 @@
+import asyncio
 import os
 import time
 import logging
+from functools import partial
 import yaml
 from typing import Callable
 
-
 from config_operator.config_source.exceptions import UnreadableFileException
 
 logging.basicConfig(level=logging.DEBUG)
@@ -16,12 +17,14 @@ LISTEN_FOR_UPDATE_INTERVAL_SECONDS = 1
 class LocalDirConfig:
 
     def __init__(self, local_dir: str,
-                 update_every_seconds: int = LISTEN_FOR_UPDATE_INTERVAL_SECONDS):
+                 update_every_seconds: float = LISTEN_FOR_UPDATE_INTERVAL_SECONDS,
+                 update_date_fun=os.path.getmtime):
         logger.info(f'create config on local dir {local_dir}')
         self._local_dir = local_dir
+        self._update_date_fun = update_date_fun
+        self._update_every_seconds = update_every_seconds
         self._latest_update = self._get_latest_update()
-        self._update_every_seconds=update_every_seconds
-        
+
     def get_files(self):
         files = []
         for f in os.listdir(self._local_dir):
@@ -49,28 +52,29 @@ class LocalDirConfig:
             raise UnreadableFileException(e)
         except yaml.parser.ParserError as e:
             raise UnreadableFileException(e)
-
+        except yaml.scanner.ScannerError as e:
+            raise UnreadableFileException(e)
 
     def _get_latest_update(self):
-        m_times = [os.path.getmtime(root) for root, _, _ in os.walk(self._local_dir)]
+        m_times = [self._update_date_fun(root) for root, _, _ in os.walk(self._local_dir)]
         if m_times:
-            return time.ctime(max(m_times))
+            return max(m_times)
         else:
             return None
 
-    def when_updated(self, callback: Callable[[], None]):
+    async def when_updated(self, callback: Callable[[], None], loop=None):
         """
           call function callback when the local directory is updated.
         """
-        while True:
-            time.sleep(self._update_every_seconds)
-            latest_update = self._get_latest_update()
-            if latest_update is None or (latest_update > self._latest_update):
-                logger.info("local config dir has been updated")
-                callback()
-                self._latest_update = latest_update
-            else:
-                logger.debug("local config dir has not been updated")
+        if loop is None:
+            loop = asyncio.get_running_loop()
 
-        return None
+        latest_update = self._get_latest_update()
+        if latest_update is None or (latest_update > self._latest_update):
+            logger.info("local config dir has been updated")
+            callback()
+        else:
+            logger.debug("local config dir has not been updated")
+        loop.call_later(self._update_every_seconds, partial(self.when_updated, callback, loop))
 
+        return None
diff --git a/config_operator/config_operator/config_source/RemoteGitConfig.py b/config_operator/config_operator/config_source/RemoteGitConfig.py
index 17d8223..38cbe2d 100644
--- a/config_operator/config_operator/config_source/RemoteGitConfig.py
+++ b/config_operator/config_operator/config_source/RemoteGitConfig.py
@@ -1,9 +1,12 @@
+import asyncio
 import logging
 import os
 import sys
-import time
-from git import Repo
+from functools import partial
 from typing import Callable
+
+from git import Repo
+
 from .LocalDirConfig import LocalDirConfig
 
 logging.basicConfig(level=logging.DEBUG)
@@ -16,10 +19,11 @@ DEFAULT_LOCAL_REPO_DIR = os.path.join(sys.prefix, 'sdap', 'conf')
 class RemoteGitConfig(LocalDirConfig):
     def __init__(self, git_url: str,
                  git_branch: str = 'master',
+                 git_username: str = None,
                  git_token: str = None,
-                 update_every_seconds: int = LISTEN_FOR_UPDATE_INTERVAL_SECONDS,
-                 local_dir: str = DEFAULT_LOCAL_REPO_DIR
-                 ):
+                 update_every_seconds: float = LISTEN_FOR_UPDATE_INTERVAL_SECONDS,
+                 local_dir: str = DEFAULT_LOCAL_REPO_DIR,
+                 repo: Repo = None):
         """
 
         :param git_url:
@@ -27,14 +31,23 @@ class RemoteGitConfig(LocalDirConfig):
         :param git_token:
         """
         self._git_url = git_url if git_url.endswith(".git") else git_url + '.git'
+        if git_username and git_token:
+            self._git_url.replace('https://', f'https://{git_username}:{git_token}')
+            self._git_url.replace('http://', f'http://{git_username}:{git_token}')
+
         self._git_branch = git_branch
         self._git_token = git_token
         if local_dir is None:
             local_dir = DEFAULT_LOCAL_REPO_DIR
         self._update_every_seconds = update_every_seconds
         super().__init__(local_dir, update_every_seconds=self._update_every_seconds)
-        self._repo = None
-        self._init_local_config_repo()
+
+        if repo:
+            self._repo = repo
+        else:
+            self._repo = None
+            self._init_local_config_repo()
+
         self._latest_commit_key = self._pull_remote()
 
     def _pull_remote(self):
@@ -49,19 +62,21 @@ class RemoteGitConfig(LocalDirConfig):
         self._repo.git.fetch()
         self._repo.git.checkout(self._git_branch)
 
-    def when_updated(self, callback: Callable[[], None]):
+    async def when_updated(self, callback: Callable[[], None], loop=None):
         """
         call function callback when the remote git repository is updated.
         """
-        while True:
-            time.sleep(self._update_every_seconds)
-            remote_commit_key = self._pull_remote()
-            if remote_commit_key != self._latest_commit_key:
-                logger.info("remote git repository has been updated")
-                callback()
-                self._latest_commit_key = remote_commit_key
-            else:
-                logger.debug("remote git repository has not been updated")
+        if loop is None:
+            loop = asyncio.get_running_loop()
 
-        return None
+        remote_commit_key = self._pull_remote()
+        if remote_commit_key != self._latest_commit_key:
+            logger.info("remote git repository has been updated")
+            callback()
+            self._latest_commit_key = remote_commit_key
+        else:
+            logger.debug("remote git repository has not been updated")
 
+        loop.call_later(self._update_every_seconds, partial(self.when_updated, callback, loop))
+
+        return None
diff --git a/config_operator/config_operator/k8s/K8sConfigMap.py b/config_operator/config_operator/k8s/K8sConfigMap.py
index e2a7a10..40d07c9 100644
--- a/config_operator/config_operator/k8s/K8sConfigMap.py
+++ b/config_operator/config_operator/k8s/K8sConfigMap.py
@@ -4,7 +4,8 @@ from kubernetes import client, config
 from config_operator.config_source import LocalDirConfig, RemoteGitConfig
 from kubernetes.client.rest import ApiException
 from typing import Union
-
+from kubernetes.client.api.core_v1_api import CoreV1Api
+from kubernetes.client import ApiClient
 from config_operator.config_source.exceptions import UnreadableFileException
 
 logging.basicConfig(level=logging.INFO)
@@ -14,19 +15,24 @@ logger = logging.getLogger(__name__)
 class K8sConfigMap:
     def __init__(self, configmap_name: str,
                  namespace: str,
-                 external_config: Union[LocalDirConfig, RemoteGitConfig]):
+                 external_config: Union[LocalDirConfig, RemoteGitConfig],
+                 api_instance: ApiClient = None,
+                 api_core_v1_instance: CoreV1Api = None):
         self._git_remote_config = external_config
         self._namespace = namespace
         self._configmap_name = configmap_name
 
-        # test is this runs inside kubernetes cluster
-        if os.getenv('KUBERNETES_SERVICE_HOST'):
-            config.load_incluster_config()
-        else:
-            config.load_kube_config()
-        configuration = client.Configuration()
-        self._api_instance = client.ApiClient(configuration)
-        self._api_core_v1_instance = client.CoreV1Api(self._api_instance)
+        if api_core_v1_instance is None:
+            # test is this runs inside kubernetes cluster
+            if os.getenv('KUBERNETES_SERVICE_HOST'):
+                config.load_incluster_config()
+            else:
+                config.load_kube_config()
+            configuration = client.Configuration()
+            api_instance = client.ApiClient(configuration)
+            api_core_v1_instance = client.CoreV1Api(api_instance)
+        self._api_instance = api_instance
+        self._api_core_v1_instance = api_core_v1_instance
         self.publish()
 
     def __del__(self):
diff --git a/config_operator/config_operator/main.py b/config_operator/config_operator/main.py
index db4dbcb..fac6741 100644
--- a/config_operator/config_operator/main.py
+++ b/config_operator/config_operator/main.py
@@ -1,4 +1,5 @@
 import logging
+import asyncio
 import kopf
 from config_operator.config_source import RemoteGitConfig
 from config_operator.k8s import K8sConfigMap
@@ -6,10 +7,10 @@ from config_operator.k8s import K8sConfigMap
 logging.basicConfig(level=logging.INFO)
 logger = logging.getLogger(__name__)
 
+
 @kopf.on.create('sdap.apache.org', 'v1', 'gitbasedconfigs')
 def create_fn(body, spec, **kwargs):
     # Get info from Git Repo Config object
-    name = body['metadata']['name']
     namespace = body['metadata']['namespace']
 
     if 'git-url' not in spec.keys():
@@ -23,7 +24,7 @@ def create_fn(body, spec, **kwargs):
     logger.info(f'config-map = {config_map}')
 
     _kargs = {}
-    for k in {'git-branch', 'git-token', 'update-every-seconds'}:
+    for k in {'git-branch', 'git-username', 'git-token', 'update-every-seconds'}:
         if k in spec:
             logger.info(f'{k} = {spec[k]}')
             _kargs[k.replace('-', '_')] = spec[k]
@@ -32,7 +33,12 @@ def create_fn(body, spec, **kwargs):
 
     config_map = K8sConfigMap(config_map, namespace, config)
 
-    config.when_updated(config_map.publish)
+    asyncio.run(config.when_updated(config_map.publish))
 
     msg = f"configmap {config_map} created from git repo {git_url}"
     return {'message': msg}
+
+
+@kopf.on.login()
+def login_fn(**kwargs):
+    return kopf.login_via_client(**kwargs)
diff --git a/config_operator/containers/k8s/git-repo-test.yml b/config_operator/containers/k8s/git-repo-test.yml
new file mode 100644
index 0000000..6a98454
--- /dev/null
+++ b/config_operator/containers/k8s/git-repo-test.yml
@@ -0,0 +1,9 @@
+apiVersion: sdap.apache.org/v1
+kind: gitBasedConfig
+metadata:
+  name: collections-config-gitcfg
+spec:
+  git-url: https://github.com/tloubrieu-jpl/sdap-ingester-config.git
+  git-branch: master
+  git-token: whatever
+  config-map: my-configmap
\ No newline at end of file
diff --git a/config_operator/requirements.txt b/config_operator/requirements.txt
index 5d452e2..84ac622 100644
--- a/config_operator/requirements.txt
+++ b/config_operator/requirements.txt
@@ -2,3 +2,4 @@ GitPython==3.1.2
 kubernetes==11.0
 kopf==0.26
 
+
diff --git a/config_operator/tests/config_source/test_LocalDirConfig.py b/config_operator/tests/config_source/test_LocalDirConfig.py
new file mode 100644
index 0000000..fed13c0
--- /dev/null
+++ b/config_operator/tests/config_source/test_LocalDirConfig.py
@@ -0,0 +1,72 @@
+import asyncio
+import os
+import time
+import unittest
+from datetime import datetime
+from unittest.mock import Mock
+
+from config_operator.config_source import LocalDirConfig
+from config_operator.config_source.exceptions import UnreadableFileException
+
+
+class TestLocalDirConfig(unittest.TestCase):
+    def test_get_files(self):
+        local_dir = os.path.join(os.path.dirname(__file__), "../resources/localDirTest")
+        local_dir_config = LocalDirConfig(local_dir)
+        files = local_dir_config.get_files()
+        self.assertEqual(len(files), 1)
+        self.assertEqual(files[0], 'collections.yml')
+
+    def test_get_good_file_content(self):
+        local_dir = os.path.join(os.path.dirname(__file__), "../resources/localDirTest")
+        local_dir_config = LocalDirConfig(local_dir)
+        files = local_dir_config.get_files()
+        content = local_dir_config.get_file_content(files[0])
+        self.assertEqual(content.strip(), 'test: 1')
+
+    def test_get_bad_file_content(self):
+        unreadable_file = False
+        try:
+            local_dir = os.path.join(os.path.dirname(__file__), "../resources/localDirBadTest")
+            local_dir_config = LocalDirConfig(local_dir)
+            files = local_dir_config.get_files()
+            content = local_dir_config.get_file_content(files[0])
+        except UnreadableFileException as e:
+            unreadable_file = True
+
+        finally:
+            self.assertTrue(unreadable_file)
+
+    def test_when_updated(self):
+
+        callback = Mock()
+        local_dir = os.path.join(os.path.dirname(__file__), "../resources/localDirTest")
+
+        local_dir_config = LocalDirConfig(local_dir,
+                                          update_every_seconds=0.25,
+                                          update_date_fun=lambda x: datetime.now().timestamp())
+
+        asyncio.run(local_dir_config.when_updated(callback))
+
+        time.sleep(2)
+
+        assert callback.called
+
+    def test_when_not_updated(self):
+
+        callback = Mock()
+        local_dir = os.path.join(os.path.dirname(__file__), "../resources/localDirTest")
+
+        local_dir_config = LocalDirConfig(local_dir,
+                                          update_every_seconds=0.25,
+                                          update_date_fun=lambda x: 0)
+
+        asyncio.run(local_dir_config.when_updated(callback))
+
+        time.sleep(2)
+
+        assert not callback.called
+
+
+if __name__ == '__main__':
+    unittest.main()
diff --git a/config_operator/tests/config_source/test_RemoteGitConfig.py b/config_operator/tests/config_source/test_RemoteGitConfig.py
new file mode 100644
index 0000000..52e6c9d
--- /dev/null
+++ b/config_operator/tests/config_source/test_RemoteGitConfig.py
@@ -0,0 +1,49 @@
+import asyncio
+import os
+import tempfile
+import time
+import unittest
+from unittest.mock import Mock
+
+from config_operator.config_source import RemoteGitConfig
+
+
+class TestRemoteDirConfig(unittest.TestCase):
+    _latest_commit = 0
+
+    def _get_origin(self):
+        commit = Mock()
+        commit.hexsha = self._latest_commit
+        self._latest_commit += 1
+
+        pull_result = Mock()
+        pull_result.commit = commit
+
+        return [pull_result]
+
+    def test_when_updated(self):
+        origin_branch = Mock()
+        origin_branch.pull = self._get_origin
+
+        remotes = Mock()
+        remotes.origin = origin_branch
+
+        repo = Mock()
+        repo.remotes = remotes
+
+        git_config = RemoteGitConfig(git_url='https://github.com/tloubrieu-jpl/sdap-ingester-config',
+                                     update_every_seconds=0.25,
+                                     local_dir=os.path.join(tempfile.gettempdir(), 'sdap'),
+                                     repo=repo)
+
+        callback = Mock()
+
+        asyncio.run(git_config.when_updated(callback))
+
+        time.sleep(2)
+
+        assert callback.called
+
+
+if __name__ == '__main__':
+    unittest.main()
diff --git a/config_operator/tests/k8s/test_K8sConfigMap.py b/config_operator/tests/k8s/test_K8sConfigMap.py
index 1660e69..e649fbc 100644
--- a/config_operator/tests/k8s/test_K8sConfigMap.py
+++ b/config_operator/tests/k8s/test_K8sConfigMap.py
@@ -1,11 +1,19 @@
 import unittest
+from unittest.mock import Mock
 import os
+from kubernetes.client.rest import ApiException
 
 from config_operator.k8s import K8sConfigMap
 from config_operator.config_source import RemoteGitConfig, LocalDirConfig
 
+if 'GIT_USERNAME' in os.environ:
+    GIT_USERNAME = os.environ['GIT_USERNAME']
+if 'GIT_TOKEN' in os.environ:
+    GIT_TOKEN = os.environ['GIT_TOKEN']
+
 
 class K8sConfigMapTest(unittest.TestCase):
+    @unittest.skip('requires remote git')
     def test_createconfigmapfromgit(self):
 
         remote_git_config = RemoteGitConfig("https://github.com/tloubrieu-jpl/sdap-ingester-config")
@@ -13,13 +21,52 @@ class K8sConfigMapTest(unittest.TestCase):
         config_map = K8sConfigMap('collection-ingester', 'sdap', remote_git_config)
         config_map.publish()
 
+    @unittest.skip('requires remote git')
+    def test_createconfigmapfromgit_with_token(self):
+        remote_git_config = RemoteGitConfig("https://podaac-git.jpl.nasa.gov:8443/podaac-sdap/deployment-configs.git",
+                                            git_username=GIT_USERNAME,
+                                            git_token=GIT_TOKEN)
+
+        config_map = K8sConfigMap('collection-ingester', 'sdap', remote_git_config)
+        config_map.publish()
+
+    @unittest.skip('requires kubernetes')
+    def test_createconfigmapfromlocaldir_with_k8s(self):
+        local_dir = os.path.join(os.path.dirname(os.path.abspath(__file__)),
+                                 '..',
+                                 'resources',
+                                 'localDirTest')
+        local_config = LocalDirConfig(local_dir)
+
+        config_map = K8sConfigMap('collection-ingester', 'sdap', local_config)
+        config_map.publish()
+
     def test_createconfigmapfromlocaldir(self):
         local_dir = os.path.join(os.path.dirname(os.path.abspath(__file__)),
                                  '..',
-                                 'resources')
-        remote_git_config = LocalDirConfig(local_dir)
+                                 'resources',
+                                 'localDirTest')
+        local_config = LocalDirConfig(local_dir)
 
-        config_map = K8sConfigMap('collection-ingester', 'sdap', remote_git_config)
+        api_instance = Mock()
+        api_instance.close = Mock()
+
+        api_core_v1_mock = Mock()
+        api_core_v1_mock.create_namespaced_config_map = Mock(return_value={
+            'api_version': 'v1',
+            'binary_data': None,
+            'data': {}
+        })
+        api_core_v1_mock.patch_namespaced_config_map = Mock(return_value={
+            'api_version': 'v1',
+            'binary_data': None,
+            'data': {}
+        })
+        api_core_v1_mock.create_namespaced_config_map.side_effect = Mock(side_effect=ApiException('409'))
+
+        config_map = K8sConfigMap('collection-ingester', 'sdap', local_config,
+                                  api_instance = api_instance,
+                                  api_core_v1_instance=api_core_v1_mock)
         config_map.publish()
 
 
diff --git a/config_operator/tests/resources/localDirBadTest/collections.yml b/config_operator/tests/resources/localDirBadTest/collections.yml
new file mode 100644
index 0000000..7903016
--- /dev/null
+++ b/config_operator/tests/resources/localDirBadTest/collections.yml
@@ -0,0 +1,2 @@
+test:
+test
diff --git a/config_operator/tests/resources/localDirTest/.hidden_file.txt b/config_operator/tests/resources/localDirTest/.hidden_file.txt
new file mode 100644
index 0000000..30d74d2
--- /dev/null
+++ b/config_operator/tests/resources/localDirTest/.hidden_file.txt
@@ -0,0 +1 @@
+test
\ No newline at end of file
diff --git a/config_operator/tests/resources/localDirTest/README.md b/config_operator/tests/resources/localDirTest/README.md
new file mode 100644
index 0000000..30d74d2
--- /dev/null
+++ b/config_operator/tests/resources/localDirTest/README.md
@@ -0,0 +1 @@
+test
\ No newline at end of file
diff --git a/config_operator/tests/resources/localDirTest/collections.yml b/config_operator/tests/resources/localDirTest/collections.yml
new file mode 100644
index 0000000..4857bf6
--- /dev/null
+++ b/config_operator/tests/resources/localDirTest/collections.yml
@@ -0,0 +1 @@
+test: 1

[incubator-sdap-ingester] 13/33: SDAP-266: add README note on synchronization of configmap with local path (#14)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 7590df910268c043c698691de7673525b0a0656a
Author: thomas loubrieu <60...@users.noreply.github.com>
AuthorDate: Wed Aug 5 10:38:57 2020 -0700

    SDAP-266: add README note on synchronization of configmap with local path (#14)
    
    Co-authored-by: thomas loubrieu <th...@jpl.nasa.gov>
---
 config_operator/README.md               | 14 ++++++++++++--
 config_operator/config_operator/main.py | 17 ++++++++++-------
 2 files changed, 22 insertions(+), 9 deletions(-)

diff --git a/config_operator/README.md b/config_operator/README.md
index 5f02804..d102d10 100644
--- a/config_operator/README.md
+++ b/config_operator/README.md
@@ -2,13 +2,23 @@
 
 ## Purpose
 
-Component which synchonizes local configuration in a directory, on a file system, or configuration files managed in a git repository with kubernetes configMap.
-This helps to make a configuration managed by the operators in a single place (git, host file system) available in the kubernetes cluster.
+Component which synchonizes configuration in a remote **GIT** repository with kubernetes configMap.
+This helps to make a configuration managed by the operators in a single place (git) available in the kubernetes cluster.
 
 For SDAP, it is used to make the configuration of the collections to be ingested available to the ingester service pods.
 
 The component runs as a kubernetes operator (see containerization section)
 
+To synchronize a configuration from a **local directory** on kubernetes hosts, you should use the following commands:
+
+    kubectl create configmap collections-config --from-file=/opt/sdap-collection-config/  -n <namespace> 
+    
+To update the configmap from the same directory run:
+
+    kubectl create configmap collections-config --from-file=/opt/sdap-collection-config/ -o yaml --dry-run | kubectl replace -n <namespace> -f -
+    
+
+
 # Developers
 
     git clone ...
diff --git a/config_operator/config_operator/main.py b/config_operator/config_operator/main.py
index 45d530f..fbbbe6b 100644
--- a/config_operator/config_operator/main.py
+++ b/config_operator/config_operator/main.py
@@ -1,14 +1,16 @@
 import logging
 import asyncio
 import kopf
-from config_operator.config_source import RemoteGitConfig
+from config_operator.config_source import RemoteGitConfig, LocalDirConfig
 from config_operator.k8s import K8sConfigMap
 
 logging.basicConfig(level=logging.INFO)
 logger = logging.getLogger(__name__)
 
 
-def create_config_synchronizer(spec, namespace):
+UPDATE_EVERY_SECOND_PROPERTY = 'update-every-seconds'
+
+def create_git_config_synchronizer(spec, namespace):
     if 'git-url' not in spec.keys():
         raise kopf.HandlerFatalError(f"git-url must be set.")
     if 'config-map' not in spec.keys():
@@ -20,7 +22,7 @@ def create_config_synchronizer(spec, namespace):
     logger.info(f'config-map = {config_map}')
 
     _kwargs = {}
-    for k in {'git-branch', 'git-username', 'git-token', 'update-every-seconds'}:
+    for k in {'git-branch', 'git-username', 'git-token', UPDATE_EVERY_SECOND_PROPERTY}:
         if k in spec:
             logger.info(f'{k} = {spec[k]}')
             _kwargs[k.replace('-', '_')] = spec[k]
@@ -38,24 +40,25 @@ def create_config_synchronizer(spec, namespace):
 
 @kopf.on.create('sdap.apache.org', 'v1', 'gitbasedconfigs')
 def create_fn(body, spec, **kwargs):
-    logger.info(f'sdap config operator creation')
+    logger.info(f'sdap git config operator creation')
 
     namespace = body['metadata']['namespace']
 
     msg = create_config_synchronizer(spec, namespace)
 
-    logger.info(f'sdap config operator created {msg}')
+    logger.info(f'sdap git config operator created {msg}')
 
     return {'message': msg}
 
 
+
 @kopf.on.update('sdap.apache.org', 'v1', 'gitbasedconfigs')
 def update_fn(spec, status, namespace, **kwargs):
-    logger.info(f'sdap config operator update')
+    logger.info(f'sdap git config operator update')
 
     msg = create_config_synchronizer(spec, namespace)
 
-    logger.info(f'sdap config operator updated {msg}')
+    logger.info(f'sdap local config operator updated {msg}')
 
     return {'message': msg}
 

[incubator-sdap-ingester] 07/33: SDAP-269: Switch to using aio-pika in collection-manager to maintain an asynchronous connection to RabbitMQ (#7)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit d748429329084cb19966c80393acaa4e53162391
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Mon Aug 3 12:51:56 2020 -0700

    SDAP-269: Switch to using aio-pika in collection-manager to maintain an asynchronous connection to RabbitMQ (#7)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 .gitignore                                         |  4 +-
 collection_manager/README.md                       |  7 ++-
 collection_manager/collection_manager/main.py      | 36 ++++++------
 .../services/CollectionProcessor.py                |  8 +--
 .../services/CollectionWatcher.py                  | 45 +++++++++------
 .../services/MessagePublisher.py                   | 41 +++++++-------
 collection_manager/requirements.txt                |  3 +-
 .../tests/services/test_CollectionProcessor.py     | 37 +++++++-----
 .../tests/services/test_CollectionWatcher.py       | 66 ++++++++--------------
 common/common/__init__.py                          |  0
 common/common/async_test_utils/AsyncTestUtils.py   | 28 +++++++++
 common/common/async_test_utils/__init__.py         |  1 +
 common/setup.py                                    | 21 +++++++
 granule_ingester/.gitignore                        |  9 ---
 granule_ingester/tests/writers/test_SolrStore.py   |  5 +-
 15 files changed, 178 insertions(+), 133 deletions(-)

diff --git a/.gitignore b/.gitignore
index e7c73b8..46e4151 100644
--- a/.gitignore
+++ b/.gitignore
@@ -2,7 +2,6 @@
 credentials.json
 token.pickle
 tmp
-sdap_ingest_manager.egg-info
 venv
 __pycache__/
 dist/
@@ -11,3 +10,6 @@ build/
 *.DS_Store
 .eggs
 temp/
+*.pyc
+*.vscode
+*.code-workspace
\ No newline at end of file
diff --git a/collection_manager/README.md b/collection_manager/README.md
index 9d00cbb..771f355 100644
--- a/collection_manager/README.md
+++ b/collection_manager/README.md
@@ -64,10 +64,11 @@ collections:
 
 ```
 ## Running the tests
-From `incubator-sdap-ingester/collection_manager`, run:
+From `incubator-sdap-ingester/`, run:
 
-    $ pip install pytest
-    $ pytest
+    $ cd common && python setup.py install
+    $ cd ../collection_manager && python setup.py install
+    $ pip install pytest && pytest
     
 ## Building the Docker image
 From `incubator-sdap-ingester/collection_manager`, run:
diff --git a/collection_manager/collection_manager/main.py b/collection_manager/collection_manager/main.py
index 7e72de5..cbe22f9 100644
--- a/collection_manager/collection_manager/main.py
+++ b/collection_manager/collection_manager/main.py
@@ -63,28 +63,26 @@ async def main():
             history_manager_builder = FileIngestionHistoryBuilder(history_path=options.history_path)
         else:
             history_manager_builder = SolrIngestionHistoryBuilder(solr_url=options.history_url)
-        publisher = MessagePublisher(host=options.rabbitmq_host,
-                                     username=options.rabbitmq_username,
-                                     password=options.rabbitmq_password,
-                                     queue=options.rabbitmq_queue)
-        publisher.connect()
-        collection_processor = CollectionProcessor(message_publisher=publisher,
-                                                   history_manager_builder=history_manager_builder)
-        collection_watcher = CollectionWatcher(collections_path=options.collections_path,
-                                               collection_updated_callback=collection_processor.process_collection,
-                                               granule_updated_callback=collection_processor.process_granule,
-                                               collections_refresh_interval=int(options.refresh))
+        async with MessagePublisher(host=options.rabbitmq_host,
+                                    username=options.rabbitmq_username,
+                                    password=options.rabbitmq_password,
+                                    queue=options.rabbitmq_queue) as publisher:
+            collection_processor = CollectionProcessor(message_publisher=publisher,
+                                                       history_manager_builder=history_manager_builder)
+            collection_watcher = CollectionWatcher(collections_path=options.collections_path,
+                                                   collection_updated_callback=collection_processor.process_collection,
+                                                   granule_updated_callback=collection_processor.process_granule,
+                                                   collections_refresh_interval=int(options.refresh))
 
-        collection_watcher.start_watching()
-
-        while True:
-            try:
-                await asyncio.sleep(1)
-            except KeyboardInterrupt:
-                return
+            await collection_watcher.start_watching()
+            while True:
+                try:
+                    await asyncio.sleep(1)
+                except KeyboardInterrupt:
+                    return
 
     except Exception as e:
-        logger.error(e)
+        logger.exception(e)
         return
 
 
diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index 232cdee..d790f4b 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -25,16 +25,16 @@ class CollectionProcessor:
         with open(MESSAGE_TEMPLATE, 'r') as config_template_file:
             self._config_template = config_template_file.read()
 
-    def process_collection(self, collection: Collection):
+    async def process_collection(self, collection: Collection):
         """
         Given a Collection, detect new granules that need to be ingested and publish RabbitMQ messages for each.
         :param collection: A Collection definition
         :return: None
         """
         for granule in collection.files_owned():
-            self.process_granule(granule, collection)
+            await self.process_granule(granule, collection)
 
-    def process_granule(self, granule: str, collection: Collection):
+    async def process_granule(self, granule: str, collection: Collection):
         """
         Determine whether a granule needs to be ingested, and if so publish a RabbitMQ message for it.
         :param granule: A path to a granule file
@@ -64,7 +64,7 @@ class CollectionProcessor:
             return
 
         dataset_config = self._fill_template(granule, collection, config_template=self._config_template)
-        self._publisher.publish_message(body=dataset_config, priority=use_priority)
+        await self._publisher.publish_message(body=dataset_config, priority=use_priority)
         history_manager.push(granule)
 
     @staticmethod
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index 2387016..8911806 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -2,9 +2,7 @@ import asyncio
 import logging
 import os
 from collections import defaultdict
-from functools import partial
-from typing import Dict, Callable, Set, Optional
-
+from typing import Dict, Callable, Set, Optional, Awaitable
 import yaml
 from watchdog.events import FileSystemEventHandler
 from watchdog.observers import Observer
@@ -22,8 +20,8 @@ logger.setLevel(logging.DEBUG)
 class CollectionWatcher:
     def __init__(self,
                  collections_path: str,
-                 collection_updated_callback: Callable[[Collection], any],
-                 granule_updated_callback: Callable[[str, Collection], any],
+                 collection_updated_callback: Callable[[Collection], Awaitable],
+                 granule_updated_callback: Callable[[str, Collection], Awaitable],
                  collections_refresh_interval: float = 30):
         if not os.path.isabs(collections_path):
             raise RelativePathError("Collections config  path must be an absolute path.")
@@ -38,7 +36,7 @@ class CollectionWatcher:
 
         self._granule_watches = set()
 
-    def start_watching(self, loop: Optional[asyncio.AbstractEventLoop] = None):
+    async def start_watching(self, loop: Optional[asyncio.AbstractEventLoop] = None):
         """
         Periodically load the Collections Configuration file to check for changes,
         and observe filesystem events for added/modified granules. When an event occurs,
@@ -46,7 +44,9 @@ class CollectionWatcher:
         :return: None
         """
 
-        self._run_periodically(loop, self._collections_refresh_interval, self._reload_and_reschedule)
+        await self._run_periodically(loop=loop,
+                                     wait_time=self._collections_refresh_interval,
+                                     func=self._reload_and_reschedule)
         self._observer.start()
 
     def collections(self) -> Set[Collection]:
@@ -99,11 +99,11 @@ class CollectionWatcher:
         self._load_collections()
         return self.collections() - old_collections
 
-    def _reload_and_reschedule(self):
+    async def _reload_and_reschedule(self):
         try:
             updated_collections = self._get_updated_collections()
             for collection in updated_collections:
-                self._collection_updated_callback(collection)
+                await self._collection_updated_callback(collection)
             if len(updated_collections) > 0:
                 self._unschedule_watches()
                 self._schedule_watches()
@@ -117,7 +117,9 @@ class CollectionWatcher:
 
     def _schedule_watches(self):
         for directory, collections in self._collections_by_dir.items():
-            granule_event_handler = _GranuleEventHandler(self._granule_updated_callback, collections)
+            granule_event_handler = _GranuleEventHandler(asyncio.get_running_loop(),
+                                                         self._granule_updated_callback,
+                                                         collections)
             # Note: the Watchdog library does not schedule a new watch
             # if one is already scheduled for the same directory
             try:
@@ -127,18 +129,23 @@ class CollectionWatcher:
                 logger.error(f"Granule directory {directory} does not exist. Ignoring {bad_collection_names}.")
 
     @classmethod
-    def _run_periodically(cls, loop: Optional[asyncio.AbstractEventLoop], wait_time: float, func: Callable, *args):
+    async def _run_periodically(cls,
+                                loop: Optional[asyncio.AbstractEventLoop],
+                                wait_time: float,
+                                func: Callable[[any], Awaitable],
+                                *args,
+                                **kwargs):
         """
         Call a function periodically. This uses asyncio, and is non-blocking.
         :param loop: An optional event loop to use. If None, the current running event loop will be used.
         :param wait_time: seconds to wait between iterations of func
-        :param func: the function that will be run
+        :param func: the async function that will be awaited
         :param args: any args that need to be provided to func
         """
         if loop is None:
             loop = asyncio.get_running_loop()
-        func(*args)
-        loop.call_later(wait_time, partial(cls._run_periodically, loop, wait_time, func))
+        await func(*args, **kwargs)
+        loop.call_later(wait_time, loop.create_task, cls._run_periodically(loop, wait_time, func, *args, **kwargs))
 
 
 class _GranuleEventHandler(FileSystemEventHandler):
@@ -146,7 +153,11 @@ class _GranuleEventHandler(FileSystemEventHandler):
     EventHandler that watches for new or modified granule files.
     """
 
-    def __init__(self, callback: Callable[[str, Collection], any], collections_for_dir: Set[Collection]):
+    def __init__(self,
+                 loop: asyncio.AbstractEventLoop,
+                 callback: Callable[[str, Collection], Awaitable],
+                 collections_for_dir: Set[Collection]):
+        self._loop = loop
         self._callback = callback
         self._collections_for_dir = collections_for_dir
 
@@ -154,7 +165,7 @@ class _GranuleEventHandler(FileSystemEventHandler):
         super().on_created(event)
         for collection in self._collections_for_dir:
             if collection.owns_file(event.src_path):
-                self._callback(event.src_path, collection)
+                self._loop.create_task(self._callback(event.src_path, collection))
 
     def on_modified(self, event):
         super().on_modified(event)
@@ -163,4 +174,4 @@ class _GranuleEventHandler(FileSystemEventHandler):
 
         for collection in self._collections_for_dir:
             if collection.owns_file(event.src_path):
-                self._callback(event.src_path, collection)
+                self._loop.create_task(self._callback(event.src_path, collection))
diff --git a/collection_manager/collection_manager/services/MessagePublisher.py b/collection_manager/collection_manager/services/MessagePublisher.py
index 559a69d..75803d1 100644
--- a/collection_manager/collection_manager/services/MessagePublisher.py
+++ b/collection_manager/collection_manager/services/MessagePublisher.py
@@ -1,4 +1,5 @@
-import pika
+from aio_pika import Message, DeliveryMode, Connection, Channel, connect_robust
+from tenacity import retry, stop_after_attempt, wait_fixed
 
 
 class MessagePublisher:
@@ -6,34 +7,34 @@ class MessagePublisher:
     def __init__(self, host: str, username: str, password: str, queue: str):
         self._connection_string = f"amqp://{username}:{password}@{host}/"
         self._queue = queue
-        self._channel = None
-        self._connection = None
+        self._channel: Channel = None
+        self._connection: Connection = None
 
-    def connect(self):
+    async def __aenter__(self):
+        await self._connect()
+        return self
+
+    async def __aexit__(self, exc_type, exc_val, exc_tb):
+        if self._connection:
+            await self._connection.close()
+
+    async def _connect(self):
         """
         Establish a connection to RabbitMQ.
         :return: None
         """
-        parameters = pika.URLParameters(self._connection_string)
-        self._connection = pika.BlockingConnection(parameters)
-        self._channel = self._connection.channel()
-        self._channel.queue_declare(self._queue, durable=True)
+        self._connection = await connect_robust(self._connection_string)
+        self._channel = await self._connection.channel()
+        await self._channel.declare_queue(self._queue, durable=True)
 
-    def publish_message(self, body: str, priority: int = None):
+    @retry(wait=wait_fixed(5), reraise=True, stop=stop_after_attempt(4))
+    async def publish_message(self, body: str, priority: int = None):
         """
         Publish a message to RabbitMQ using the optional message priority.
         :param body: A string to publish to RabbitMQ
         :param priority: An optional integer priority to use for the message
         :return: None
         """
-        properties = pika.BasicProperties(content_type='text/plain',
-                                          delivery_mode=1,
-                                          priority=priority)
-        self._channel.basic_publish(exchange='',
-                                    routing_key=self._queue,
-                                    body=body,
-                                    properties=properties)
-
-    def __del__(self):
-        if self._connection:
-            self._connection.close()
+        message = Message(body=body.encode('utf-8'), priority=priority, delivery_mode=DeliveryMode.PERSISTENT)
+        await self._channel.default_exchange.publish(message, routing_key=self._queue)
+
diff --git a/collection_manager/requirements.txt b/collection_manager/requirements.txt
index f16bde3..7e47c51 100644
--- a/collection_manager/requirements.txt
+++ b/collection_manager/requirements.txt
@@ -1,6 +1,7 @@
 PyYAML==5.3.1
 pystache==0.5.4
 pysolr==3.8.1
-pika==1.1.0
 watchdog==0.10.2
 requests==2.23.0
+aio-pika==6.6.1
+tenacity==6.2.0
\ No newline at end of file
diff --git a/collection_manager/tests/services/test_CollectionProcessor.py b/collection_manager/tests/services/test_CollectionProcessor.py
index 56d5393..aa143f3 100644
--- a/collection_manager/tests/services/test_CollectionProcessor.py
+++ b/collection_manager/tests/services/test_CollectionProcessor.py
@@ -6,6 +6,7 @@ from collection_manager.entities import Collection
 from collection_manager.services import CollectionProcessor
 from collection_manager.services.history_manager import FileIngestionHistoryBuilder
 from collection_manager.services.history_manager import GranuleStatus
+from common.async_test_utils import AsyncMock, async_test
 
 
 class TestCollectionProcessor(unittest.TestCase):
@@ -63,10 +64,11 @@ class TestCollectionProcessor(unittest.TestCase):
         filled = CollectionProcessor._fill_template("/granules/test_granule.nc", collection, template)
         self.assertEqual(filled, expected)
 
+    @async_test
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
-    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
-    def test_process_granule_with_historical_granule(self, mock_publisher, mock_history_builder, mock_history):
+    @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
+    async def test_process_granule_with_historical_granule(self, mock_publisher, mock_history_builder, mock_history):
         mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_HISTORICAL
         mock_history_builder.build.return_value = mock_history
 
@@ -79,15 +81,17 @@ class TestCollectionProcessor(unittest.TestCase):
                                 date_from=None,
                                 date_to=None)
 
-        collection_processor.process_granule("test.nc", collection)
+        await collection_processor.process_granule("test.nc", collection)
 
         mock_publisher.publish_message.assert_called_with(body=mock.ANY, priority=1)
         mock_history.push.assert_called()
 
+    @async_test
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
-    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
-    def test_process_granule_with_forward_processing_granule(self, mock_publisher, mock_history_builder, mock_history):
+    @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
+    async def test_process_granule_with_forward_processing_granule(self, mock_publisher, mock_history_builder,
+                                                                   mock_history):
         mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_FORWARD_PROCESSING
         mock_history_builder.build.return_value = mock_history
 
@@ -100,15 +104,16 @@ class TestCollectionProcessor(unittest.TestCase):
                                 date_from=None,
                                 date_to=None)
 
-        collection_processor.process_granule("test.h5", collection)
+        await collection_processor.process_granule("test.h5", collection)
 
         mock_publisher.publish_message.assert_called_with(body=mock.ANY, priority=2)
         mock_history.push.assert_called()
 
+    @async_test
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
-    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
-    def test_process_granule_with_forward_processing_granule_and_no_priority(self, mock_publisher,
+    @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
+    async def test_process_granule_with_forward_processing_granule_and_no_priority(self, mock_publisher,
                                                                              mock_history_builder, mock_history):
         mock_history.get_granule_status.return_value = GranuleStatus.DESIRED_FORWARD_PROCESSING
         mock_history_builder.build.return_value = mock_history
@@ -121,15 +126,16 @@ class TestCollectionProcessor(unittest.TestCase):
                                 date_from=None,
                                 date_to=None)
 
-        collection_processor.process_granule("test.h5", collection)
+        await collection_processor.process_granule("test.h5", collection)
 
         mock_publisher.publish_message.assert_called_with(body=mock.ANY, priority=1)
         mock_history.push.assert_called()
 
+    @async_test
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
-    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
-    def test_process_granule_with_undesired_granule(self, mock_publisher, mock_history_builder, mock_history):
+    @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
+    async def test_process_granule_with_undesired_granule(self, mock_publisher, mock_history_builder, mock_history):
         mock_history.get_granule_status.return_value = GranuleStatus.UNDESIRED
         mock_history_builder.build.return_value = mock_history
 
@@ -142,15 +148,16 @@ class TestCollectionProcessor(unittest.TestCase):
                                 date_from=None,
                                 date_to=None)
 
-        collection_processor.process_granule("test.nc", collection)
+        await collection_processor.process_granule("test.nc", collection)
 
         mock_publisher.publish_message.assert_not_called()
         mock_history.push.assert_not_called()
 
+    @async_test
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistory', autospec=True)
     @mock.patch('collection_manager.services.history_manager.FileIngestionHistoryBuilder', autospec=True)
-    @mock.patch('collection_manager.services.MessagePublisher', autospec=True)
-    def test_process_granule_with_unsupported_file_type(self, mock_publisher, mock_history_builder, mock_history):
+    @mock.patch('collection_manager.services.MessagePublisher', new_callable=AsyncMock)
+    async def test_process_granule_with_unsupported_file_type(self, mock_publisher, mock_history_builder, mock_history):
         mock_history_builder.build.return_value = mock_history
 
         collection_processor = CollectionProcessor(mock_publisher, mock_history_builder)
@@ -162,7 +169,7 @@ class TestCollectionProcessor(unittest.TestCase):
                                 date_from=None,
                                 date_to=None)
 
-        collection_processor.process_granule("test.foo", collection)
+        await collection_processor.process_granule("test.foo", collection)
 
         mock_publisher.publish_message.assert_not_called()
         mock_history.push.assert_not_called()
diff --git a/collection_manager/tests/services/test_CollectionWatcher.py b/collection_manager/tests/services/test_CollectionWatcher.py
index b954812..14e7c3c 100644
--- a/collection_manager/tests/services/test_CollectionWatcher.py
+++ b/collection_manager/tests/services/test_CollectionWatcher.py
@@ -1,4 +1,3 @@
-import asyncio
 import os
 import tempfile
 import unittest
@@ -9,6 +8,7 @@ from collection_manager.entities import Collection
 from collection_manager.entities.exceptions import CollectionConfigParsingError, CollectionConfigFileNotFoundError, \
     RelativePathCollectionError, ConflictingPathCollectionError
 from collection_manager.services import CollectionWatcher
+from common.async_test_utils.AsyncTestUtils import AsyncAssert, AsyncMock, async_test
 
 
 class TestCollectionWatcher(unittest.TestCase):
@@ -30,7 +30,7 @@ class TestCollectionWatcher(unittest.TestCase):
 
     def test_load_collections_loads_all_collections(self):
         collections_path = os.path.join(os.path.dirname(__file__), '../resources/collections.yml')
-        collection_watcher = CollectionWatcher(collections_path, Mock(), Mock())
+        collection_watcher = CollectionWatcher(collections_path, AsyncMock(), AsyncMock())
         collection_watcher._load_collections()
 
         self.assertEqual(len(collection_watcher._collections_by_dir), 2)
@@ -120,7 +120,8 @@ class TestCollectionWatcher(unittest.TestCase):
                                 date_to=None)
         self.assertRaises(ConflictingPathCollectionError, collection_watcher._validate_collection, collection)
 
-    def test_collection_callback_is_called(self):
+    @async_test
+    async def test_collection_callback_is_called(self):
         collections_config = tempfile.NamedTemporaryFile("w+b", buffering=0, delete=False)
         granule_dir = tempfile.TemporaryDirectory()
         collections_str = f"""collections:
@@ -131,14 +132,13 @@ class TestCollectionWatcher(unittest.TestCase):
   forward-processing-priority: 5"""
         collections_config.write(collections_str.encode("utf-8"))
 
-        collection_callback = Mock()
+        collection_callback = AsyncMock()
         collection_watcher = CollectionWatcher(collections_path=collections_config.name,
                                                collection_updated_callback=collection_callback,
-                                               granule_updated_callback=Mock(),
+                                               granule_updated_callback=AsyncMock(),
                                                collections_refresh_interval=0.1)
 
-        loop = asyncio.new_event_loop()
-        collection_watcher.start_watching(loop)
+        await collection_watcher.start_watching()
 
         collections_str = f"""
 - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
@@ -149,14 +149,14 @@ class TestCollectionWatcher(unittest.TestCase):
         """
         collections_config.write(collections_str.encode("utf-8"))
 
-        loop.run_until_complete(self.assert_called_within_timeout(collection_callback, call_count=2))
+        await AsyncAssert.assert_called_within_timeout(collection_callback, call_count=2)
 
-        loop.close()
         collections_config.close()
         granule_dir.cleanup()
         os.remove(collections_config.name)
 
-    def test_granule_callback_is_called_on_new_file(self):
+    @async_test
+    async def test_granule_callback_is_called_on_new_file(self):
         with tempfile.NamedTemporaryFile("w+b", buffering=0) as collections_config:
             granule_dir = tempfile.TemporaryDirectory()
             collections_str = f"""
@@ -169,21 +169,18 @@ collections:
             """
             collections_config.write(collections_str.encode("utf-8"))
 
-            granule_callback = Mock()
-            collection_watcher = CollectionWatcher(collections_config.name, Mock(), granule_callback)
-
-            loop = asyncio.new_event_loop()
-            collection_watcher.start_watching(loop)
+            granule_callback = AsyncMock()
+            collection_watcher = CollectionWatcher(collections_config.name, AsyncMock(), granule_callback)
 
+            await collection_watcher.start_watching()
             new_granule = open(os.path.join(granule_dir.name, 'test.nc'), "w+")
+            await AsyncAssert.assert_called_within_timeout(granule_callback)
 
-            loop.run_until_complete(self.assert_called_within_timeout(granule_callback))
-
-            loop.close()
             new_granule.close()
             granule_dir.cleanup()
 
-    def test_granule_callback_is_called_on_modified_file(self):
+    @async_test
+    async def test_granule_callback_is_called_on_modified_file(self):
         with tempfile.NamedTemporaryFile("w+b", buffering=0) as collections_config:
             granule_dir = tempfile.TemporaryDirectory()
             collections_str = f"""
@@ -197,33 +194,20 @@ collections:
             collections_config.write(collections_str.encode("utf-8"))
             new_granule = open(os.path.join(granule_dir.name, 'test.nc'), "w+")
 
-            granule_callback = Mock()
-            collection_watcher = CollectionWatcher(collections_config.name, Mock(), granule_callback)
+            granule_callback = AsyncMock()
+            collection_watcher = CollectionWatcher(collections_config.name, AsyncMock(), granule_callback)
 
-            loop = asyncio.new_event_loop()
-            collection_watcher.start_watching(loop)
+            await collection_watcher.start_watching()
 
             new_granule.write("hello world")
             new_granule.close()
 
-            loop.run_until_complete(self.assert_called_within_timeout(granule_callback))
-            loop.close()
+            await AsyncAssert.assert_called_within_timeout(granule_callback)
             granule_dir.cleanup()
 
-    def test_run_periodically(self):
-        callback = Mock()
-        loop = asyncio.new_event_loop()
-        CollectionWatcher._run_periodically(loop, 0.1, callback)
-        loop.run_until_complete(self.assert_called_within_timeout(callback, timeout_sec=0.3, call_count=2))
-        loop.close()
-
-    @staticmethod
-    async def assert_called_within_timeout(mock_func, timeout_sec=1.0, call_count=1):
-        start = datetime.now()
-
-        while (datetime.now() - start).total_seconds() < timeout_sec:
-            await asyncio.sleep(0.01)
-            if mock_func.call_count >= call_count:
-                return
-        raise AssertionError(f"{mock_func} did not reach {call_count} calls called within {timeout_sec} sec")
+    @async_test
+    async def test_run_periodically(self):
+        callback = AsyncMock()
+        await CollectionWatcher._run_periodically(None, 0.1, callback)
+        await AsyncAssert.assert_called_within_timeout(callback, timeout_sec=0.3, call_count=2)
 
diff --git a/common/common/__init__.py b/common/common/__init__.py
new file mode 100644
index 0000000..e69de29
diff --git a/common/common/async_test_utils/AsyncTestUtils.py b/common/common/async_test_utils/AsyncTestUtils.py
new file mode 100644
index 0000000..ccb829b
--- /dev/null
+++ b/common/common/async_test_utils/AsyncTestUtils.py
@@ -0,0 +1,28 @@
+import asyncio
+from datetime import datetime
+from unittest import mock
+
+
+class AsyncMock(mock.MagicMock):
+    async def __call__(self, *args, **kwargs):
+        return super().__call__(*args, **kwargs)
+
+
+class AsyncAssert:
+    @staticmethod
+    async def assert_called_within_timeout(mock_func, timeout_sec=1.0, call_count=1):
+        start = datetime.now()
+
+        while (datetime.now() - start).total_seconds() < timeout_sec:
+            await asyncio.sleep(0.01)
+            if mock_func.call_count >= call_count:
+                return
+        raise AssertionError(f"Mock did not reach {call_count} calls called within {timeout_sec} sec")
+
+
+def async_test(coro):
+    def wrapper(*args, **kwargs):
+        loop = asyncio.new_event_loop()
+        return loop.run_until_complete(coro(*args, **kwargs))
+
+    return wrapper
diff --git a/common/common/async_test_utils/__init__.py b/common/common/async_test_utils/__init__.py
new file mode 100644
index 0000000..12563af
--- /dev/null
+++ b/common/common/async_test_utils/__init__.py
@@ -0,0 +1 @@
+from .AsyncTestUtils import AsyncMock, AsyncAssert, async_test
diff --git a/common/setup.py b/common/setup.py
new file mode 100644
index 0000000..ed621c8
--- /dev/null
+++ b/common/setup.py
@@ -0,0 +1,21 @@
+import re
+
+import setuptools
+
+PACKAGE_NAME = "sdap_ingester_common"
+
+setuptools.setup(
+    name=PACKAGE_NAME,
+    author="Apache - SDAP",
+    author_email="dev@sdap.apache.org",
+    description="a module of common functions for the sdap ingester components",
+    url="https://github.com/apache/incubator-sdap-ingester",
+    packages=setuptools.find_packages(),
+    classifiers=[
+        "Programming Language :: Python :: 3",
+        "Operating System :: OS Independent",
+        "Development Status :: 4 - Beta",
+    ],
+    python_requires='>=3.7',
+    include_package_data=True
+)
diff --git a/granule_ingester/.gitignore b/granule_ingester/.gitignore
deleted file mode 100644
index 5408b74..0000000
--- a/granule_ingester/.gitignore
+++ /dev/null
@@ -1,9 +0,0 @@
-.vscode
-.idea
-*.egg-info
-*__pycache__
-*.pytest_cache
-*.code-workspace
-.DS_STORE
-build
-dist
\ No newline at end of file
diff --git a/granule_ingester/tests/writers/test_SolrStore.py b/granule_ingester/tests/writers/test_SolrStore.py
index 76b85ac..0e971ce 100644
--- a/granule_ingester/tests/writers/test_SolrStore.py
+++ b/granule_ingester/tests/writers/test_SolrStore.py
@@ -1,4 +1,3 @@
-import asyncio
 import unittest
 
 from nexusproto import DataTile_pb2 as nexusproto
@@ -43,8 +42,8 @@ class TestSolrStore(unittest.TestCase):
         self.assertEqual('test_variable', solr_doc['tile_var_name_s'])
         self.assertAlmostEqual(-90.5, solr_doc['tile_min_lon'])
         self.assertAlmostEqual(90.0, solr_doc['tile_max_lon'])
-        self.assertAlmostEqual(-180.1, solr_doc['tile_min_lat'])
-        self.assertAlmostEqual(180.2, solr_doc['tile_max_lat'])
+        self.assertAlmostEqual(-180.1, solr_doc['tile_min_lat'], delta=1E-5)
+        self.assertAlmostEqual(180.2, solr_doc['tile_max_lat'], delta=1E-5)
         self.assertEqual('1992-01-01T00:00:00Z', solr_doc['tile_min_time_dt'])
         self.assertEqual('1992-01-02T00:00:00Z', solr_doc['tile_max_time_dt'])
         self.assertAlmostEqual(-10.0, solr_doc['tile_min_val_d'])

[incubator-sdap-ingester] 22/33: SDAP-288: S3 ingestion support (#24)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 31bff5e6c524c213ae33fe1af39f18f4da2969cc
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Thu Nov 12 14:59:34 2020 -0800

    SDAP-288: S3 ingestion support (#24)
---
 .../collection_manager/entities/Collection.py      |  33 +++--
 .../collection_manager/entities/__init__.py        |   1 +
 collection_manager/collection_manager/main.py      |  22 +++-
 .../services/CollectionProcessor.py                |  28 ++--
 .../services/CollectionWatcher.py                  |  90 ++++++++-----
 .../collection_manager/services/S3Observer.py      | 145 +++++++++++++++++++++
 .../collection_manager/services/__init__.py        |   1 +
 .../history_manager/FileIngestionHistory.py        |   3 +-
 .../services/history_manager/IngestionHistory.py   |  79 ++++++-----
 .../history_manager/SolrIngestionHistory.py        |   9 +-
 collection_manager/docker/Dockerfile               |   4 +-
 collection_manager/docker/entrypoint.sh            |   5 +-
 collection_manager/requirements.txt                |   6 +-
 collection_manager/setup.py                        |   2 +-
 .../history_manager/test_FileIngestionHistory.py   |   6 +-
 .../tests/services/test_S3Observer.py              |   8 ++
 granule_ingester/conda-requirements.txt            |   1 -
 granule_ingester/docker/Dockerfile                 |   2 +
 granule_ingester/requirements.txt                  |   5 +-
 19 files changed, 334 insertions(+), 116 deletions(-)

diff --git a/collection_manager/collection_manager/entities/Collection.py b/collection_manager/collection_manager/entities/Collection.py
index 031a3a9..7a45b66 100644
--- a/collection_manager/collection_manager/entities/Collection.py
+++ b/collection_manager/collection_manager/entities/Collection.py
@@ -1,13 +1,20 @@
 import os
+from urllib.parse import urlparse
 from dataclasses import dataclass
 from datetime import datetime
 from fnmatch import fnmatch
 from glob import glob
 from typing import List, Optional
+from enum import Enum
 
 from collection_manager.entities.exceptions import MissingValueCollectionError
 
 
+class CollectionStorageType(Enum):
+    LOCAL = 1
+    S3 = 2
+
+
 @dataclass(frozen=True)
 class Collection:
     dataset_id: str
@@ -39,20 +46,28 @@ class Collection:
         except KeyError as e:
             raise MissingValueCollectionError(missing_value=e.args[0])
 
+    def storage_type(self):
+        if urlparse(self.path).scheme == 's3':
+            return CollectionStorageType.S3
+        else:
+            return CollectionStorageType.LOCAL
+
     def directory(self):
-        if os.path.isdir(self.path):
+        if urlparse(self.path).scheme == 's3':
+            return self.path
+        elif os.path.isdir(self.path):
             return self.path
         else:
             return os.path.dirname(self.path)
 
     def owns_file(self, file_path: str) -> bool:
-        if os.path.isdir(file_path):
-            raise IsADirectoryError()
-
-        if os.path.isdir(self.path):
-            return os.path.dirname(file_path) == self.path
+        if urlparse(file_path).scheme == 's3':
+            return file_path.find(self.path) == 0
         else:
-            return fnmatch(file_path, self.path)
+            if os.path.isdir(file_path):
+                raise IsADirectoryError()
 
-    def files_owned(self) -> List[str]:
-        return glob(self.path, recursive=True)
+            if os.path.isdir(self.path):
+                return os.path.dirname(file_path) == self.path
+            else:
+                return fnmatch(file_path, self.path)
diff --git a/collection_manager/collection_manager/entities/__init__.py b/collection_manager/collection_manager/entities/__init__.py
index 165341b..b9c7a25 100644
--- a/collection_manager/collection_manager/entities/__init__.py
+++ b/collection_manager/collection_manager/entities/__init__.py
@@ -1 +1,2 @@
 from .Collection import Collection
+from .Collection import CollectionStorageType
\ No newline at end of file
diff --git a/collection_manager/collection_manager/main.py b/collection_manager/collection_manager/main.py
index cbe22f9..b80ae7c 100644
--- a/collection_manager/collection_manager/main.py
+++ b/collection_manager/collection_manager/main.py
@@ -3,8 +3,11 @@ import asyncio
 import logging
 import os
 
-from collection_manager.services import CollectionProcessor, CollectionWatcher, MessagePublisher
-from collection_manager.services.history_manager import SolrIngestionHistoryBuilder, FileIngestionHistoryBuilder
+from collection_manager.services import (CollectionProcessor,
+                                         CollectionWatcher, MessagePublisher)
+from collection_manager.services.history_manager import (
+    FileIngestionHistoryBuilder, SolrIngestionHistoryBuilder,
+    md5sum_from_filepath)
 
 logging.basicConfig(level=logging.INFO)
 logging.getLogger("pika").setLevel(logging.WARNING)
@@ -51,6 +54,9 @@ def get_args() -> argparse.Namespace:
                         default='30',
                         metavar="INTERVAL",
                         help='Number of seconds after which to reload the collections config file. (Default: 30)')
+    parser.add_argument('--s3-bucket',
+                        metavar='S3-BUCKET',
+                        help='Optional name of an AWS S3 bucket where granules are stored. If this option is set, then all collections to be scanned must have their granules on S3, not the local filesystem.')
 
     return parser.parse_args()
 
@@ -59,10 +65,14 @@ async def main():
     try:
         options = get_args()
 
+        signature_fun = None if options.s3_bucket else md5sum_from_filepath
+
         if options.history_path:
-            history_manager_builder = FileIngestionHistoryBuilder(history_path=options.history_path)
+            history_manager_builder = FileIngestionHistoryBuilder(history_path=options.history_path,
+                                                                  signature_fun=signature_fun)
         else:
-            history_manager_builder = SolrIngestionHistoryBuilder(solr_url=options.history_url)
+            history_manager_builder = SolrIngestionHistoryBuilder(solr_url=options.history_url,
+                                                                  signature_fun=signature_fun)
         async with MessagePublisher(host=options.rabbitmq_host,
                                     username=options.rabbitmq_username,
                                     password=options.rabbitmq_password,
@@ -70,9 +80,9 @@ async def main():
             collection_processor = CollectionProcessor(message_publisher=publisher,
                                                        history_manager_builder=history_manager_builder)
             collection_watcher = CollectionWatcher(collections_path=options.collections_path,
-                                                   collection_updated_callback=collection_processor.process_collection,
                                                    granule_updated_callback=collection_processor.process_granule,
-                                                   collections_refresh_interval=int(options.refresh))
+                                                   collections_refresh_interval=int(options.refresh),
+                                                   s3_bucket=options.s3_bucket)
 
             await collection_watcher.start_watching()
             while True:
diff --git a/collection_manager/collection_manager/services/CollectionProcessor.py b/collection_manager/collection_manager/services/CollectionProcessor.py
index 975f50c..96c461e 100644
--- a/collection_manager/collection_manager/services/CollectionProcessor.py
+++ b/collection_manager/collection_manager/services/CollectionProcessor.py
@@ -1,12 +1,16 @@
 import logging
 import os.path
+from glob import glob
 from typing import Dict
-import yaml
+from datetime import datetime
 
+import yaml
 from collection_manager.entities import Collection
 from collection_manager.services import MessagePublisher
-from collection_manager.services.history_manager import IngestionHistory, GranuleStatus
-from collection_manager.services.history_manager.IngestionHistory import IngestionHistoryBuilder
+from collection_manager.services.history_manager import (GranuleStatus,
+                                                         IngestionHistory)
+from collection_manager.services.history_manager.IngestionHistory import \
+    IngestionHistoryBuilder
 
 logger = logging.getLogger(__name__)
 
@@ -20,16 +24,7 @@ class CollectionProcessor:
         self._history_manager_builder = history_manager_builder
         self._history_manager_cache: Dict[str, IngestionHistory] = {}
 
-    async def process_collection(self, collection: Collection):
-        """
-        Given a Collection, detect new granules that need to be ingested and publish RabbitMQ messages for each.
-        :param collection: A Collection definition
-        :return: None
-        """
-        for granule in collection.files_owned():
-            await self.process_granule(granule, collection)
-
-    async def process_granule(self, granule: str, collection: Collection):
+    async def process_granule(self, granule: str, modified_time: int, collection: Collection):
         """
         Determine whether a granule needs to be ingested, and if so publish a RabbitMQ message for it.
         :param granule: A path to a granule file
@@ -40,7 +35,10 @@ class CollectionProcessor:
             return
 
         history_manager = self._get_history_manager(collection.dataset_id)
-        granule_status = await history_manager.get_granule_status(granule, collection.date_from, collection.date_to)
+        granule_status = await history_manager.get_granule_status(granule,
+                                                                  modified_time,
+                                                                  collection.date_from,
+                                                                  collection.date_to)
 
         if granule_status is GranuleStatus.DESIRED_FORWARD_PROCESSING:
             logger.info(f"New granule '{granule}' detected for forward-processing ingestion "
@@ -60,7 +58,7 @@ class CollectionProcessor:
 
         dataset_config = self._generate_ingestion_message(granule, collection)
         await self._publisher.publish_message(body=dataset_config, priority=use_priority)
-        await history_manager.push(granule)
+        await history_manager.push(granule, modified_time)
 
     @staticmethod
     def _file_supported(file_path: str):
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index 1c7c1be..b1aaf4e 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -1,17 +1,22 @@
 import asyncio
+from datetime import datetime
+from collection_manager.entities.Collection import CollectionStorageType, Collection
+from collection_manager.services.S3Observer import S3Event, S3Observer
 import logging
 import os
 import time
 from collections import defaultdict
-from typing import Awaitable, Callable, Dict, Optional, Set
+from glob import glob
+from typing import Awaitable, Callable, Dict, List, Optional, Set
 
 import yaml
-from collection_manager.entities import Collection
-from collection_manager.entities.exceptions import (
-    CollectionConfigFileNotFoundError, CollectionConfigParsingError,
-    ConflictingPathCollectionError, MissingValueCollectionError,
-    RelativePathCollectionError, RelativePathError)
-from watchdog.events import FileSystemEventHandler
+from collection_manager.entities.exceptions import (CollectionConfigFileNotFoundError,
+                                                    CollectionConfigParsingError,
+                                                    ConflictingPathCollectionError,
+                                                    MissingValueCollectionError,
+                                                    RelativePathCollectionError,
+                                                    RelativePathError)
+from watchdog.events import FileCreatedEvent, FileModifiedEvent, FileSystemEventHandler
 from watchdog.observers.polling import PollingObserver as Observer
 
 logger = logging.getLogger(__name__)
@@ -21,19 +26,18 @@ logger.setLevel(logging.DEBUG)
 class CollectionWatcher:
     def __init__(self,
                  collections_path: str,
-                 collection_updated_callback: Callable[[Collection], Awaitable],
                  granule_updated_callback: Callable[[str, Collection], Awaitable],
+                 s3_bucket: Optional[str] = None,
                  collections_refresh_interval: float = 30):
         if not os.path.isabs(collections_path):
             raise RelativePathError("Collections config  path must be an absolute path.")
 
         self._collections_path = collections_path
-        self._collection_updated_callback = collection_updated_callback
         self._granule_updated_callback = granule_updated_callback
         self._collections_refresh_interval = collections_refresh_interval
 
         self._collections_by_dir: Dict[str, Set[Collection]] = defaultdict(set)
-        self._observer = Observer()
+        self._observer = S3Observer(s3_bucket, initial_scan=True) if s3_bucket else Observer()
 
         self._granule_watches = set()
 
@@ -48,7 +52,11 @@ class CollectionWatcher:
         await self._run_periodically(loop=loop,
                                      wait_time=self._collections_refresh_interval,
                                      func=self._reload_and_reschedule)
-        self._observer.start()
+
+        if isinstance(self._observer, S3Observer):
+            await self._observer.start()
+        else:
+            self._observer.start()
 
     def _collections(self) -> Set[Collection]:
         """
@@ -58,11 +66,15 @@ class CollectionWatcher:
         return {collection for collections in self._collections_by_dir.values() for collection in collections}
 
     def _validate_collection(self, collection: Collection):
-        directory = collection.directory()
-        if not os.path.isabs(directory):
-            raise RelativePathCollectionError(collection=collection)
-        if directory == os.path.dirname(self._collections_path):
-            raise ConflictingPathCollectionError(collection=collection)
+        if collection.storage_type() == CollectionStorageType.S3:
+            # do some S3 path validation here
+            return
+        else:
+            directory = collection.directory()
+            if not os.path.isabs(directory):
+                raise RelativePathCollectionError(collection=collection)
+            if directory == os.path.dirname(self._collections_path):
+                raise ConflictingPathCollectionError(collection=collection)
 
     def _load_collections(self):
         try:
@@ -100,15 +112,23 @@ class CollectionWatcher:
         self._load_collections()
         return self._collections() - old_collections
 
+    async def _call_callback_for_all_granules(self, collections: List[Collection]):
+        logger.info(f"Scanning files for {len(collections)} collections...")
+        start = time.perf_counter()
+        for collection in collections:
+            for granule_path in glob(collection.path, recursive=True):
+                modified_time = int(os.path.getmtime(granule_path))
+                await self._granule_updated_callback(granule_path, modified_time, collection)
+        logger.info(f"Finished scanning files in {time.perf_counter() - start} seconds.")
+
     async def _reload_and_reschedule(self):
         try:
             updated_collections = self._get_updated_collections()
             if len(updated_collections) > 0:
-                logger.info(f"Scanning files for {len(updated_collections)} collections...")
-                start = time.perf_counter()
-                for collection in updated_collections:
-                    await self._collection_updated_callback(collection)
-                logger.info(f"Finished scanning files in {time.perf_counter() - start} seconds.")
+                # For S3 collections, the S3Observer will report as new any files that haven't already been scanned.
+                # So we only need to rescan granules here if not using S3.
+                if not isinstance(self._observer, S3Observer):
+                    await self._call_callback_for_all_granules(collections=updated_collections)
 
                 self._unschedule_watches()
                 self._schedule_watches()
@@ -128,7 +148,10 @@ class CollectionWatcher:
             # Note: the Watchdog library does not schedule a new watch
             # if one is already scheduled for the same directory
             try:
-                self._granule_watches.add(self._observer.schedule(granule_event_handler, directory, recursive=True))
+                if isinstance(self._observer, S3Observer):
+                    self._granule_watches.add(self._observer.schedule(granule_event_handler, directory))
+                else:
+                    self._granule_watches.add(self._observer.schedule(granule_event_handler, directory, recursive=True))
             except (FileNotFoundError, NotADirectoryError):
                 bad_collection_names = ' and '.join([col.dataset_id for col in collections])
                 logger.error(f"Granule directory {directory} does not exist. Ignoring {bad_collection_names}.")
@@ -168,18 +191,21 @@ class _GranuleEventHandler(FileSystemEventHandler):
 
     def on_created(self, event):
         super().on_created(event)
-        for collection in self._collections_for_dir:
-            try:
-                if collection.owns_file(event.src_path):
-                    self._loop.create_task(self._callback(event.src_path, collection))
-            except IsADirectoryError:
-                pass
+        self._handle_event(event)
 
     def on_modified(self, event):
         super().on_modified(event)
-        if os.path.isdir(event.src_path):
-            return
+        self._handle_event(event)
 
+    def _handle_event(self, event):
+        path = event.src_path
         for collection in self._collections_for_dir:
-            if collection.owns_file(event.src_path):
-                self._loop.create_task(self._callback(event.src_path, collection))
+            try:
+                if collection.owns_file(path):
+                    if isinstance(event, S3Event):
+                        modified_time = int(event.modified_time.timestamp())
+                    else:
+                        modified_time = int(os.path.getmtime(path))
+                    self._loop.create_task(self._callback(path, modified_time, collection))
+            except IsADirectoryError:
+                return
diff --git a/collection_manager/collection_manager/services/S3Observer.py b/collection_manager/collection_manager/services/S3Observer.py
new file mode 100644
index 0000000..87458a9
--- /dev/null
+++ b/collection_manager/collection_manager/services/S3Observer.py
@@ -0,0 +1,145 @@
+import asyncio
+from urllib.parse import urlparse
+import datetime
+import os
+import time
+from dataclasses import dataclass
+from typing import Set, Dict, Optional, Callable, Awaitable
+
+import aioboto3
+
+
+@dataclass
+class S3Event:
+    src_path: str
+    modified_time: datetime.datetime
+
+
+class S3FileModifiedEvent(S3Event):
+    pass
+
+
+class S3FileCreatedEvent(S3Event):
+    pass
+
+
+class S3Watch(object):
+    def __init__(self, path: str, event_handler) -> None:
+        self.path = path
+        self.event_handler = event_handler
+
+
+class S3Observer:
+
+    def __init__(self, bucket, initial_scan=False) -> None:
+        self._bucket = bucket
+        self._cache: Dict[str, datetime.datetime] = {}
+        self._initial_scan = initial_scan
+        self._watches: Set[S3Watch] = set()
+
+        self._has_polled = False
+
+    async def start(self):
+        await self._run_periodically(loop=None,
+                                     wait_time=30,
+                                     func=self._poll)
+
+    def unschedule(self, watch: S3Watch):
+        self._watches.remove(watch)
+
+    def schedule(self, event_handler, path: str):
+        watch = S3Watch(path=path, event_handler=event_handler)
+        self._watches.add(watch)
+        return watch
+
+    @classmethod
+    async def _run_periodically(cls,
+                                loop: Optional[asyncio.AbstractEventLoop],
+                                wait_time: float,
+                                func: Callable[[any], Awaitable],
+                                *args,
+                                **kwargs):
+        """
+        Call a function periodically. This uses asyncio, and is non-blocking.
+        :param loop: An optional event loop to use. If None, the current running event loop will be used.
+        :param wait_time: seconds to wait between iterations of func
+        :param func: the async function that will be awaited
+        :param args: any args that need to be provided to func
+        """
+        if loop is None:
+            loop = asyncio.get_running_loop()
+        await func(*args, **kwargs)
+        loop.call_later(wait_time, loop.create_task, cls._run_periodically(loop, wait_time, func, *args, **kwargs))
+
+    async def _poll(self):
+        new_cache = {}
+        watch_index = {}
+
+        for watch in self._watches:
+            new_cache_for_watch = await self._get_s3_files(watch.path)
+            new_index = {file: watch for file in new_cache_for_watch}
+
+            new_cache = {**new_cache, **new_cache_for_watch}
+            watch_index = {**watch_index, **new_index}
+        difference = set(new_cache.items()) - set(self._cache.items())
+
+        if self._has_polled or self._initial_scan:
+            for (file, modified_date) in difference:
+                watch = watch_index[file]
+                file_is_new = file not in self._cache
+
+                if file_is_new:
+                    watch.event_handler.on_created(S3FileCreatedEvent(src_path=file, modified_time=modified_date))
+                else:
+                    watch.event_handler.on_modified(S3FileModifiedEvent(src_path=file, modified_time=modified_date))
+
+        self._cache = new_cache
+        self._has_polled = True
+
+    async def _get_s3_files(self, path: str):
+        new_cache = {}
+
+        start = time.perf_counter()
+        async with aioboto3.resource("s3") as s3:
+            bucket = await s3.Bucket(self._bucket)
+
+            object_key = S3Observer._get_object_key(path)
+            async for file in bucket.objects.filter(Prefix=object_key):
+                new_cache[f"s3://{file.bucket_name}/{file.key}"] = await file.last_modified
+        end = time.perf_counter()
+        duration = end - start
+
+        print(f"Retrieved {len(new_cache)} objects in {duration}")
+
+        return new_cache
+
+    def _get_object_key(full_path: str):
+        key = urlparse(full_path).path.strip("/")
+        return key
+
+
+async def test():
+    observer = S3Observer(bucket="nexus-ingest", initial_scan=False)
+    handler = Handler()
+    observer.schedule(handler, 'avhrr/2012')
+    observer.schedule(handler, 'avhrr/2013')
+
+    await observer.start()
+
+    while True:
+        try:
+            await asyncio.sleep(1)
+        except KeyboardInterrupt:
+            return
+
+
+class Handler:
+    def on_created(self, event: S3Event):
+        print(f"File created: {event.src_path}")
+
+    def on_modified(self, event: S3Event):
+        print(f"File modified: {event.src_path}")
+
+
+if __name__ == "__main__":
+    asyncio.run(test())
diff --git a/collection_manager/collection_manager/services/__init__.py b/collection_manager/collection_manager/services/__init__.py
index 635d3dc..553e1b7 100644
--- a/collection_manager/collection_manager/services/__init__.py
+++ b/collection_manager/collection_manager/services/__init__.py
@@ -16,3 +16,4 @@
 from .CollectionProcessor import CollectionProcessor
 from .CollectionWatcher import CollectionWatcher
 from .MessagePublisher import MessagePublisher
+from .S3Observer import S3Observer
diff --git a/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
index ffa065f..cf92997 100644
--- a/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/FileIngestionHistory.py
@@ -4,7 +4,6 @@ from pathlib import Path
 
 from collection_manager.services.history_manager.IngestionHistory import IngestionHistory
 from collection_manager.services.history_manager.IngestionHistory import IngestionHistoryBuilder
-from collection_manager.services.history_manager.IngestionHistory import md5sum_from_filepath
 
 logger = logging.getLogger(__name__)
 
@@ -33,7 +32,7 @@ class FileIngestionHistory(IngestionHistory):
         """
         self._dataset_id = dataset_id
         self._history_file_path = os.path.join(history_path, f'{dataset_id}.csv')
-        self._signature_fun = md5sum_from_filepath if signature_fun is None else signature_fun
+        self._signature_fun = signature_fun
         self._history_dict = {}
         self._load_history_dict()
 
diff --git a/collection_manager/collection_manager/services/history_manager/IngestionHistory.py b/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
index ef73ccb..7f33c79 100644
--- a/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/IngestionHistory.py
@@ -1,4 +1,5 @@
 import hashlib
+from urllib.parse import urlparse
 import logging
 import os
 from abc import ABC, abstractmethod
@@ -6,6 +7,8 @@ from datetime import datetime
 from enum import Enum
 from typing import Optional
 
+from botocore.compat import filter_ssl_warnings
+
 logger = logging.getLogger(__name__)
 
 BLOCK_SIZE = 65536
@@ -35,49 +38,28 @@ class GranuleStatus(Enum):
 
 class IngestionHistory(ABC):
     _signature_fun = None
-    _latest_ingested_file_update = None
+    _latest_ingested_file_update: int = None
 
-    async def push(self, file_path: str):
+    async def push(self, file_path: str, modified_timestamp: int):
         """
         Record a file as having been ingested.
         :param file_path: The full path to the file to record.
         :return: None
         """
-        file_path = file_path.strip()
-        file_name = os.path.basename(file_path)
-        signature = self._signature_fun(file_path)
+        file_name = IngestionHistory._get_standardized_path(file_path)
+        signature = self._signature_fun(file_path) if self._signature_fun else str(modified_timestamp)
         await self._push_record(file_name, signature)
 
         if not self._latest_ingested_file_update:
-            self._latest_ingested_file_update = os.path.getmtime(file_path)
+            self._latest_ingested_file_update = modified_timestamp
         else:
-            self._latest_ingested_file_update = max(self._latest_ingested_file_update, os.path.getmtime(file_path))
+            self._latest_ingested_file_update = max(self._latest_ingested_file_update, modified_timestamp)
 
         await self._save_latest_timestamp()
 
-    def latest_ingested_mtime(self) -> Optional[datetime]:
-        """
-        Return the modified time of the most recently modified file that was ingested.
-        :return: A datetime or None
-        """
-        if self._latest_ingested_file_update:
-            return datetime.fromtimestamp(self._latest_ingested_file_update)
-        else:
-            return None
-
-    async def already_ingested(self, file_path: str) -> bool:
-        """
-        Return a boolean indicating whether the specified file has already been ingested, based on its signature.
-        :param file_path: The full path of a file to search for in the history.
-        :return: A boolean indicating whether this file has already been ingested or not
-        """
-        file_path = file_path.strip()
-        file_name = os.path.basename(file_path)
-        signature = self._signature_fun(file_path)
-        return signature == await self._get_signature(file_name)
-
     async def get_granule_status(self,
                                  file_path: str,
+                                 modified_timestamp: int,
                                  date_from: datetime = None,
                                  date_to: datetime = None) -> GranuleStatus:
         """
@@ -94,13 +76,43 @@ class IngestionHistory(ABC):
                         should fall in order to be "desired".
         :return: A GranuleStatus enum.
         """
-        if self._in_time_range(file_path, date_from=self.latest_ingested_mtime()):
+        signature = self._signature_fun(file_path) if self._signature_fun else str(modified_timestamp)
+
+        if self._in_time_range(modified_timestamp, start_date=self._latest_ingested_mtime()):
             return GranuleStatus.DESIRED_FORWARD_PROCESSING
-        elif self._in_time_range(file_path, date_from, date_to) and not await self.already_ingested(file_path):
+        elif self._in_time_range(modified_timestamp, date_from, date_to) and not await self._already_ingested(file_path, signature):
             return GranuleStatus.DESIRED_HISTORICAL
         else:
             return GranuleStatus.UNDESIRED
 
+    def _get_standardized_path(file_path: str):
+        file_path = file_path.strip()
+        # TODO: Why do we need to record the basename of the path, instead of just the full path?
+        # The only reason this is here right now is for backwards compatibility to avoid triggering a full reingestion.
+        if urlparse(file_path).scheme == 's3':
+            return urlparse(file_path).path.strip("/")
+        else:
+            return os.path.basename(file_path)
+
+    def _latest_ingested_mtime(self) -> Optional[datetime]:
+        """
+        Return the modified time of the most recently modified file that was ingested.
+        :return: A datetime or None
+        """
+        if self._latest_ingested_file_update:
+            return datetime.fromtimestamp(self._latest_ingested_file_update)
+        else:
+            return None
+
+    async def _already_ingested(self, file_path: str, signature) -> bool:
+        """
+        Return a boolean indicating whether the specified file has already been ingested, based on its signature.
+        :param file_path: The full path of a file to search for in the history.
+        :return: A boolean indicating whether this file has already been ingested or not
+        """
+        file_name = IngestionHistory._get_standardized_path(file_path)
+        return signature == await self._get_signature(file_name)
+
     @abstractmethod
     async def _save_latest_timestamp(self):
         pass
@@ -114,15 +126,14 @@ class IngestionHistory(ABC):
         pass
 
     @staticmethod
-    def _in_time_range(file, date_from: datetime = None, date_to: datetime = None):
+    def _in_time_range(timestamp: int, start_date: datetime = None, end_date: datetime = None):
         """
         :param file: file path as a string
         :param date_from: timestamp, can be None
         :param date_to: timestamp, can be None
         :return: True is the update time of the file is between ts_from and ts_to. False otherwise
         """
-        file_modified_time = os.path.getmtime(file)
-        is_after_from = date_from.timestamp() < file_modified_time if date_from else True
-        is_before_to = date_to.timestamp() > file_modified_time if date_to else True
+        is_after_from = int(start_date.timestamp()) < timestamp if start_date else True
+        is_before_to = int(end_date.timestamp()) > timestamp if end_date else True
 
         return is_after_from and is_before_to
diff --git a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
index 59f5cd7..c6d26a5 100644
--- a/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
+++ b/collection_manager/collection_manager/services/history_manager/SolrIngestionHistory.py
@@ -3,11 +3,8 @@ import logging
 
 import pysolr
 import requests
-
+from collection_manager.services.history_manager.IngestionHistory import (IngestionHistory, IngestionHistoryBuilder)
 from common.async_utils.AsyncUtils import run_in_executor
-from collection_manager.services.history_manager.IngestionHistory import IngestionHistory
-from collection_manager.services.history_manager.IngestionHistory import IngestionHistoryBuilder
-from collection_manager.services.history_manager.IngestionHistory import md5sum_from_filepath
 
 logging.getLogger("pysolr").setLevel(logging.WARNING)
 logger = logging.getLogger(__name__)
@@ -40,7 +37,7 @@ class SolrIngestionHistory(IngestionHistory):
             self._solr_granules = pysolr.Solr(f"{self._url_prefix}/{self._granule_collection_name}")
             self._solr_datasets = pysolr.Solr(f"{self._url_prefix}/{self._dataset_collection_name}")
             self._dataset_id = dataset_id
-            self._signature_fun = md5sum_from_filepath if signature_fun is None else signature_fun
+            self._signature_fun = signature_fun
             self._latest_ingested_file_update = self._get_latest_file_update()
         except requests.exceptions.RequestException:
             raise DatasetIngestionHistorySolrException(f"solr instance unreachable {solr_url}")
@@ -67,7 +64,7 @@ class SolrIngestionHistory(IngestionHistory):
             self._solr_datasets.add([{
                 'id': self._dataset_id,
                 'dataset_s': self._dataset_id,
-                'latest_update_l': int(self._latest_ingested_file_update)}])
+                'latest_update_l': self._latest_ingested_file_update}])
             self._solr_datasets.commit()
 
     def _get_latest_file_update(self):
diff --git a/collection_manager/docker/Dockerfile b/collection_manager/docker/Dockerfile
index 2a57784..83e94ad 100644
--- a/collection_manager/docker/Dockerfile
+++ b/collection_manager/docker/Dockerfile
@@ -12,7 +12,9 @@ COPY collection_manager/requirements.txt /collection_manager/requirements.txt
 COPY collection_manager/README.md /collection_manager/README.md
 COPY collection_manager/docker/entrypoint.sh /entrypoint.sh
 
-RUN cd /common && python setup.py install
+RUN cd /common && python setup.py install 
 RUN cd /collection_manager && python setup.py install
 
+RUN pip install boto3==1.16.10
+
 ENTRYPOINT ["/bin/bash", "/entrypoint.sh"]
diff --git a/collection_manager/docker/entrypoint.sh b/collection_manager/docker/entrypoint.sh
index 988dd2c..cad304a 100644
--- a/collection_manager/docker/entrypoint.sh
+++ b/collection_manager/docker/entrypoint.sh
@@ -7,5 +7,6 @@ python /collection_manager/collection_manager/main.py \
   $([[ ! -z "$RABBITMQ_PASSWORD" ]] && echo --rabbitmq-password=$RABBITMQ_PASSWORD) \
   $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq-queue=$RABBITMQ_QUEUE) \
   $([[ ! -z "$HISTORY_URL" ]] && echo --history-url=$HISTORY_URL) \
-  $([[ ! -z "$HISTORY_PATH" ]] && echo --history-path=$HISTORY_PATH)
-  $([[ ! -z "$REFRESH" ]] && echo --refresh=$REFRESH)
+  $([[ ! -z "$HISTORY_PATH" ]] && echo --history-path=$HISTORY_PATH) \
+  $([[ ! -z "$REFRESH" ]] && echo --refresh=$REFRESH) \
+  $([[ ! -z "$S3_BUCKET" ]] && echo --s3-bucket=$S3_BUCKET)
diff --git a/collection_manager/requirements.txt b/collection_manager/requirements.txt
index ee12c89..c4b6323 100644
--- a/collection_manager/requirements.txt
+++ b/collection_manager/requirements.txt
@@ -3,5 +3,7 @@ pystache==0.5.4
 pysolr==3.9.0
 watchdog==0.10.2
 requests==2.23.0
-aio-pika==6.6.1
-tenacity==6.2.0
\ No newline at end of file
+tenacity==6.2.0
+aioboto3==8.0.5
+aiohttp==3.7.2
+aio-pika==6.7.1
\ No newline at end of file
diff --git a/collection_manager/setup.py b/collection_manager/setup.py
index 0616d0f..e1178f8 100644
--- a/collection_manager/setup.py
+++ b/collection_manager/setup.py
@@ -29,7 +29,7 @@ setuptools.setup(
         "Operating System :: OS Independent",
         "Development Status :: 4 - Beta",
     ],
-    python_requires='>=3.6',
+    python_requires='>=3.8',
     include_package_data=True,
     install_requires=pip_requirements
 )
diff --git a/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py b/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
index 07ab0e1..8bd939e 100644
--- a/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
+++ b/collection_manager/tests/services/history_manager/test_FileIngestionHistory.py
@@ -36,7 +36,7 @@ class TestFileIngestionHistory(unittest.TestCase):
             # history_manager with this file
             current_file_path = pathlib.Path(__file__)
             await ingestion_history.push(str(current_file_path))
-            self.assertTrue(await ingestion_history.already_ingested(str(current_file_path)))
+            self.assertTrue(await ingestion_history._already_ingested(str(current_file_path)))
 
             del ingestion_history
 
@@ -47,7 +47,7 @@ class TestFileIngestionHistory(unittest.TestCase):
             # history_manager with this file
             current_file_path = pathlib.Path(__file__)
             await ingestion_history.push(str(current_file_path))
-            self.assertTrue(await ingestion_history.already_ingested(str(current_file_path)))
+            self.assertTrue(await ingestion_history._already_ingested(str(current_file_path)))
 
             del ingestion_history
 
@@ -57,7 +57,7 @@ class TestFileIngestionHistory(unittest.TestCase):
             ingestion_history = FileIngestionHistory(history_dir, DATASET_ID, md5sum_from_filepath)
             # history_manager with this file
             current_file_path = pathlib.Path(__file__)
-            self.assertFalse(await ingestion_history.already_ingested(str(current_file_path)))
+            self.assertFalse(await ingestion_history._already_ingested(str(current_file_path)))
 
 
 if __name__ == '__main__':
diff --git a/collection_manager/tests/services/test_S3Observer.py b/collection_manager/tests/services/test_S3Observer.py
new file mode 100644
index 0000000..3fa49e0
--- /dev/null
+++ b/collection_manager/tests/services/test_S3Observer.py
@@ -0,0 +1,8 @@
+from collection_manager.services import S3Observer
+import unittest
+
+
+class TestS3Observer(unittest.TestCase):
+
+    def test_get_object_key(self):
+        self.assertEqual('test_dir/object.nc', S3Observer._get_object_key('s3://test-bucket/test_dir/object.nc'))
diff --git a/granule_ingester/conda-requirements.txt b/granule_ingester/conda-requirements.txt
index da92b1e..810e278 100644
--- a/granule_ingester/conda-requirements.txt
+++ b/granule_ingester/conda-requirements.txt
@@ -7,5 +7,4 @@ xarray
 pyyaml==5.3.1
 requests==2.23.0
 aiohttp==3.6.2
-aio-pika==6.6.1
 tenacity
diff --git a/granule_ingester/docker/Dockerfile b/granule_ingester/docker/Dockerfile
index 57bacff..1e7aedd 100644
--- a/granule_ingester/docker/Dockerfile
+++ b/granule_ingester/docker/Dockerfile
@@ -18,6 +18,8 @@ RUN ./install_nexusproto.sh
 RUN cd /common && python setup.py install
 RUN cd /sdap && python setup.py install
 
+RUN pip install boto3==1.16.10
+
 RUN apk del .build-deps
 
 ENTRYPOINT ["/bin/sh", "/entrypoint.sh"]
\ No newline at end of file
diff --git a/granule_ingester/requirements.txt b/granule_ingester/requirements.txt
index 9b06860..d82e6ce 100644
--- a/granule_ingester/requirements.txt
+++ b/granule_ingester/requirements.txt
@@ -1,6 +1,7 @@
 cassandra-driver==3.23.0
 aiomultiprocess==0.7.0
-aioboto3
+aioboto3==8.0.5
 tblib==1.6.0
 pysolr==3.9.0
-kazoo==2.8.0
\ No newline at end of file
+kazoo==2.8.0
+aio-pika==6.7.1
\ No newline at end of file

[incubator-sdap-ingester] 05/33: SDAP-259: The Collection Manager now reloads the Collections Config file on an interval instead of watching for filesystem events (#5)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 242de669a25e90fe47e80e9e8152b8f81ecdd62b
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Thu Jul 9 10:26:22 2020 -0500

    SDAP-259: The Collection Manager now reloads the Collections Config file on an interval instead of watching for filesystem events (#5)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 collection_manager/collection_manager/main.py      | 15 ++++--
 .../services/CollectionWatcher.py                  | 56 ++++++++++++----------
 collection_manager/docker/entrypoint.sh            |  1 +
 .../tests/services/test_CollectionWatcher.py       | 44 ++++++++++++-----
 4 files changed, 73 insertions(+), 43 deletions(-)

diff --git a/collection_manager/collection_manager/main.py b/collection_manager/collection_manager/main.py
index d8d2a5a..7e72de5 100644
--- a/collection_manager/collection_manager/main.py
+++ b/collection_manager/collection_manager/main.py
@@ -1,7 +1,7 @@
 import argparse
+import asyncio
 import logging
 import os
-import time
 
 from collection_manager.services import CollectionProcessor, CollectionWatcher, MessagePublisher
 from collection_manager.services.history_manager import SolrIngestionHistoryBuilder, FileIngestionHistoryBuilder
@@ -47,11 +47,15 @@ def get_args() -> argparse.Namespace:
                         default="nexus",
                         metavar="QUEUE",
                         help='Name of the RabbitMQ queue to consume from. (Default: "nexus")')
+    parser.add_argument('--refresh',
+                        default='30',
+                        metavar="INTERVAL",
+                        help='Number of seconds after which to reload the collections config file. (Default: 30)')
 
     return parser.parse_args()
 
 
-def main():
+async def main():
     try:
         options = get_args()
 
@@ -68,13 +72,14 @@ def main():
                                                    history_manager_builder=history_manager_builder)
         collection_watcher = CollectionWatcher(collections_path=options.collections_path,
                                                collection_updated_callback=collection_processor.process_collection,
-                                               granule_updated_callback=collection_processor.process_granule)
+                                               granule_updated_callback=collection_processor.process_granule,
+                                               collections_refresh_interval=int(options.refresh))
 
         collection_watcher.start_watching()
 
         while True:
             try:
-                time.sleep(1)
+                await asyncio.sleep(1)
             except KeyboardInterrupt:
                 return
 
@@ -84,4 +89,4 @@ def main():
 
 
 if __name__ == "__main__":
-    main()
+    asyncio.run(main())
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index a3c3bf7..2387016 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -1,7 +1,9 @@
+import asyncio
 import logging
 import os
 from collections import defaultdict
-from typing import Dict, Callable, Set
+from functools import partial
+from typing import Dict, Callable, Set, Optional
 
 import yaml
 from watchdog.events import FileSystemEventHandler
@@ -21,30 +23,31 @@ class CollectionWatcher:
     def __init__(self,
                  collections_path: str,
                  collection_updated_callback: Callable[[Collection], any],
-                 granule_updated_callback: Callable[[str, Collection], any]):
+                 granule_updated_callback: Callable[[str, Collection], any],
+                 collections_refresh_interval: float = 30):
         if not os.path.isabs(collections_path):
             raise RelativePathError("Collections config  path must be an absolute path.")
 
         self._collections_path = collections_path
         self._collection_updated_callback = collection_updated_callback
         self._granule_updated_callback = granule_updated_callback
+        self._collections_refresh_interval = collections_refresh_interval
 
         self._collections_by_dir: Dict[str, Set[Collection]] = defaultdict(set)
         self._observer = Observer()
 
         self._granule_watches = set()
 
-    def start_watching(self):
+    def start_watching(self, loop: Optional[asyncio.AbstractEventLoop] = None):
         """
-        Start observing filesystem events for added/modified granules or changes to the Collections configuration file.
-        When an event occurs, call the appropriate callback that was passed in during instantiation.
+        Periodically load the Collections Configuration file to check for changes,
+        and observe filesystem events for added/modified granules. When an event occurs,
+        call the appropriate callback that was passed in during instantiation.
         :return: None
         """
-        self._observer.schedule(
-            _CollectionEventHandler(file_path=self._collections_path, callback=self._reload_and_reschedule),
-            os.path.dirname(self._collections_path))
+
+        self._run_periodically(loop, self._collections_refresh_interval, self._reload_and_reschedule)
         self._observer.start()
-        self._reload_and_reschedule()
 
     def collections(self) -> Set[Collection]:
         """
@@ -98,10 +101,12 @@ class CollectionWatcher:
 
     def _reload_and_reschedule(self):
         try:
-            for collection in self._get_updated_collections():
+            updated_collections = self._get_updated_collections()
+            for collection in updated_collections:
                 self._collection_updated_callback(collection)
-            self._unschedule_watches()
-            self._schedule_watches()
+            if len(updated_collections) > 0:
+                self._unschedule_watches()
+                self._schedule_watches()
         except CollectionConfigParsingError as e:
             logger.error(e)
 
@@ -121,20 +126,19 @@ class CollectionWatcher:
                 bad_collection_names = ' and '.join([col.dataset_id for col in collections])
                 logger.error(f"Granule directory {directory} does not exist. Ignoring {bad_collection_names}.")
 
-
-class _CollectionEventHandler(FileSystemEventHandler):
-    """
-    EventHandler that watches for changes to the Collections config file.
-    """
-
-    def __init__(self, file_path: str, callback: Callable[[], any]):
-        self._callback = callback
-        self._file_path = file_path
-
-    def on_modified(self, event):
-        super().on_modified(event)
-        if event.src_path == self._file_path:
-            self._callback()
+    @classmethod
+    def _run_periodically(cls, loop: Optional[asyncio.AbstractEventLoop], wait_time: float, func: Callable, *args):
+        """
+        Call a function periodically. This uses asyncio, and is non-blocking.
+        :param loop: An optional event loop to use. If None, the current running event loop will be used.
+        :param wait_time: seconds to wait between iterations of func
+        :param func: the function that will be run
+        :param args: any args that need to be provided to func
+        """
+        if loop is None:
+            loop = asyncio.get_running_loop()
+        func(*args)
+        loop.call_later(wait_time, partial(cls._run_periodically, loop, wait_time, func))
 
 
 class _GranuleEventHandler(FileSystemEventHandler):
diff --git a/collection_manager/docker/entrypoint.sh b/collection_manager/docker/entrypoint.sh
index eb88f75..988dd2c 100644
--- a/collection_manager/docker/entrypoint.sh
+++ b/collection_manager/docker/entrypoint.sh
@@ -8,3 +8,4 @@ python /collection_manager/collection_manager/main.py \
   $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq-queue=$RABBITMQ_QUEUE) \
   $([[ ! -z "$HISTORY_URL" ]] && echo --history-url=$HISTORY_URL) \
   $([[ ! -z "$HISTORY_PATH" ]] && echo --history-path=$HISTORY_PATH)
+  $([[ ! -z "$REFRESH" ]] && echo --refresh=$REFRESH)
diff --git a/collection_manager/tests/services/test_CollectionWatcher.py b/collection_manager/tests/services/test_CollectionWatcher.py
index 8c6ab5f..b954812 100644
--- a/collection_manager/tests/services/test_CollectionWatcher.py
+++ b/collection_manager/tests/services/test_CollectionWatcher.py
@@ -1,6 +1,6 @@
+import asyncio
 import os
 import tempfile
-import time
 import unittest
 from datetime import datetime
 from unittest.mock import Mock
@@ -132,8 +132,13 @@ class TestCollectionWatcher(unittest.TestCase):
         collections_config.write(collections_str.encode("utf-8"))
 
         collection_callback = Mock()
-        collection_watcher = CollectionWatcher(collections_config.name, collection_callback, Mock())
-        collection_watcher.start_watching()
+        collection_watcher = CollectionWatcher(collections_path=collections_config.name,
+                                               collection_updated_callback=collection_callback,
+                                               granule_updated_callback=Mock(),
+                                               collections_refresh_interval=0.1)
+
+        loop = asyncio.new_event_loop()
+        collection_watcher.start_watching(loop)
 
         collections_str = f"""
 - id: TELLUS_GRACE_MASCON_CRI_GRID_RL05_V2_LAND
@@ -143,9 +148,11 @@ class TestCollectionWatcher(unittest.TestCase):
   forward-processing-priority: 5
         """
         collections_config.write(collections_str.encode("utf-8"))
-        collections_config.close()
 
-        self.assert_called_within_timeout(collection_callback, timeout_sec=1, call_count=2)
+        loop.run_until_complete(self.assert_called_within_timeout(collection_callback, call_count=2))
+
+        loop.close()
+        collections_config.close()
         granule_dir.cleanup()
         os.remove(collections_config.name)
 
@@ -164,12 +171,15 @@ collections:
 
             granule_callback = Mock()
             collection_watcher = CollectionWatcher(collections_config.name, Mock(), granule_callback)
-            collection_watcher.start_watching()
+
+            loop = asyncio.new_event_loop()
+            collection_watcher.start_watching(loop)
 
             new_granule = open(os.path.join(granule_dir.name, 'test.nc'), "w+")
 
-            self.assert_called_within_timeout(granule_callback)
+            loop.run_until_complete(self.assert_called_within_timeout(granule_callback))
 
+            loop.close()
             new_granule.close()
             granule_dir.cleanup()
 
@@ -189,21 +199,31 @@ collections:
 
             granule_callback = Mock()
             collection_watcher = CollectionWatcher(collections_config.name, Mock(), granule_callback)
-            collection_watcher.start_watching()
+
+            loop = asyncio.new_event_loop()
+            collection_watcher.start_watching(loop)
 
             new_granule.write("hello world")
             new_granule.close()
 
-            self.assert_called_within_timeout(granule_callback)
-
+            loop.run_until_complete(self.assert_called_within_timeout(granule_callback))
+            loop.close()
             granule_dir.cleanup()
 
+    def test_run_periodically(self):
+        callback = Mock()
+        loop = asyncio.new_event_loop()
+        CollectionWatcher._run_periodically(loop, 0.1, callback)
+        loop.run_until_complete(self.assert_called_within_timeout(callback, timeout_sec=0.3, call_count=2))
+        loop.close()
+
     @staticmethod
-    def assert_called_within_timeout(mock_func, timeout_sec=1.0, call_count=1):
+    async def assert_called_within_timeout(mock_func, timeout_sec=1.0, call_count=1):
         start = datetime.now()
 
         while (datetime.now() - start).total_seconds() < timeout_sec:
-            time.sleep(0.01)
+            await asyncio.sleep(0.01)
             if mock_func.call_count >= call_count:
                 return
         raise AssertionError(f"{mock_func} did not reach {call_count} calls called within {timeout_sec} sec")
+

[incubator-sdap-ingester] 24/33: SDAP-295: Include dataset name hash in tile ID generation (#25)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit aa829582fe2421218b3d86f20b965abc62953af3
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Nov 24 12:31:13 2020 -0800

    SDAP-295: Include dataset name hash in tile ID generation (#25)
---
 granule_ingester/granule_ingester/processors/GenerateTileId.py | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/granule_ingester/granule_ingester/processors/GenerateTileId.py b/granule_ingester/granule_ingester/processors/GenerateTileId.py
index 2d965f7..c7ab4d1 100644
--- a/granule_ingester/granule_ingester/processors/GenerateTileId.py
+++ b/granule_ingester/granule_ingester/processors/GenerateTileId.py
@@ -26,7 +26,9 @@ class GenerateTileId(TileProcessor):
         granule = os.path.basename(tile.summary.granule)
         variable_name = tile.summary.data_var_name
         spec = tile.summary.section_spec
-        generated_id = uuid.uuid3(uuid.NAMESPACE_DNS, granule + variable_name + spec)
+        dataset_name = tile.summary.dataset_name
+
+        generated_id = uuid.uuid3(uuid.NAMESPACE_DNS, dataset_name + granule + variable_name + spec)
 
         tile.summary.tile_id = str(generated_id)
         return tile

[incubator-sdap-ingester] 11/33: SDAP-271 Cassandra authentication support (#11)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit a9a0629405199049a822e938f46d43b575448629
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Aug 4 15:16:40 2020 -0700

    SDAP-271 Cassandra authentication support (#11)
---
 granule_ingester/docker/entrypoint.sh              |  2 ++
 .../granule_ingester/exceptions/Exceptions.py      | 41 ++++++++++++++++++++++
 .../granule_ingester/exceptions/__init__.py        | 11 ++++++
 granule_ingester/granule_ingester/main.py          | 30 ++++++++++++----
 .../granule_ingester/writers/CassandraStore.py     | 39 ++++++++++++++------
 5 files changed, 106 insertions(+), 17 deletions(-)

diff --git a/granule_ingester/docker/entrypoint.sh b/granule_ingester/docker/entrypoint.sh
index e6f7262..b703ee3 100644
--- a/granule_ingester/docker/entrypoint.sh
+++ b/granule_ingester/docker/entrypoint.sh
@@ -7,4 +7,6 @@ python /sdap/granule_ingester/main.py \
   $([[ ! -z "$RABBITMQ_QUEUE" ]] && echo --rabbitmq_queue=$RABBITMQ_QUEUE) \
   $([[ ! -z "$CASSANDRA_CONTACT_POINTS" ]] && echo --cassandra_contact_points=$CASSANDRA_CONTACT_POINTS) \
   $([[ ! -z "$CASSANDRA_PORT" ]] && echo --cassandra_port=$CASSANDRA_PORT) \
+  $([[ ! -z "$CASSANDRA_USERNAME" ]] && echo --cassandra_username=$CASSANDRA_USERNAME) \
+  $([[ ! -z "$CASSANDRA_PASSWORD" ]] && echo --cassandra_password=$CASSANDRA_PASSWORD) \
   $([[ ! -z "$SOLR_HOST_AND_PORT" ]] && echo --solr_host_and_port=$SOLR_HOST_AND_PORT)
diff --git a/granule_ingester/granule_ingester/exceptions/Exceptions.py b/granule_ingester/granule_ingester/exceptions/Exceptions.py
new file mode 100644
index 0000000..c648b99
--- /dev/null
+++ b/granule_ingester/granule_ingester/exceptions/Exceptions.py
@@ -0,0 +1,41 @@
+class PipelineBuildingError(Exception):
+    pass
+
+
+class PipelineRunningError(Exception):
+    pass
+
+
+class TileProcessingError(Exception):
+    pass
+
+
+class LostConnectionError(Exception):
+    pass
+
+
+class RabbitMQLostConnectionError(LostConnectionError):
+    pass
+
+
+class CassandraLostConnectionError(LostConnectionError):
+    pass
+
+class SolrLostConnectionError(LostConnectionError):
+    pass
+
+
+class FailedHealthCheckError(Exception):
+    pass
+
+
+class CassandraFailedHealthCheckError(FailedHealthCheckError):
+    pass
+
+
+class SolrFailedHealthCheckError(FailedHealthCheckError):
+    pass
+
+
+class RabbitMQFailedHealthCheckError(FailedHealthCheckError):
+    pass
diff --git a/granule_ingester/granule_ingester/exceptions/__init__.py b/granule_ingester/granule_ingester/exceptions/__init__.py
new file mode 100644
index 0000000..ea0969f
--- /dev/null
+++ b/granule_ingester/granule_ingester/exceptions/__init__.py
@@ -0,0 +1,11 @@
+from .Exceptions import CassandraFailedHealthCheckError
+from .Exceptions import CassandraLostConnectionError
+from .Exceptions import FailedHealthCheckError
+from .Exceptions import LostConnectionError
+from .Exceptions import PipelineBuildingError
+from .Exceptions import PipelineRunningError
+from .Exceptions import RabbitMQFailedHealthCheckError
+from .Exceptions import RabbitMQLostConnectionError
+from .Exceptions import SolrFailedHealthCheckError
+from .Exceptions import SolrLostConnectionError
+from .Exceptions import TileProcessingError
diff --git a/granule_ingester/granule_ingester/main.py b/granule_ingester/granule_ingester/main.py
index 5a8fc2d..9010e33 100644
--- a/granule_ingester/granule_ingester/main.py
+++ b/granule_ingester/granule_ingester/main.py
@@ -25,8 +25,8 @@ from granule_ingester.writers import CassandraStore
 from granule_ingester.writers import SolrStore
 
 
-def cassandra_factory(contact_points, port):
-    store = CassandraStore(contact_points, port)
+def cassandra_factory(contact_points, port, username, password):
+    store = CassandraStore(contact_points=contact_points, port=port, username=username, password=password)
     store.connect()
     return store
 
@@ -72,6 +72,14 @@ async def main():
                         default=9042,
                         metavar="PORT",
                         help='Cassandra port. (Default: 9042)')
+    parser.add_argument('--cassandra_username',
+                        metavar="USERNAME",
+                        default=None,
+                        help='Cassandra username. Optional.')
+    parser.add_argument('--cassandra_password',
+                        metavar="PASSWORD",
+                        default=None,
+                        help='Cassandra password. Optional.')
     parser.add_argument('--solr_host_and_port',
                         default='http://localhost:8983',
                         metavar='HOST:PORT',
@@ -94,6 +102,8 @@ async def main():
     config_values_str = "\n".join(["{} = {}".format(arg, getattr(args, arg)) for arg in vars(args)])
     logger.info("Using configuration values:\n{}".format(config_values_str))
 
+    cassandra_username = args.cassandra_username
+    cassandra_password = args.cassandra_password
     cassandra_contact_points = args.cassandra_contact_points
     cassandra_port = args.cassandra_port
     solr_host_and_port = args.solr_host_and_port
@@ -102,12 +112,18 @@ async def main():
                         rabbitmq_username=args.rabbitmq_username,
                         rabbitmq_password=args.rabbitmq_password,
                         rabbitmq_queue=args.rabbitmq_queue,
-                        data_store_factory=partial(cassandra_factory, cassandra_contact_points, cassandra_port),
+                        data_store_factory=partial(cassandra_factory,
+                                                   cassandra_contact_points,
+                                                   cassandra_port,
+                                                   cassandra_username,
+                                                   cassandra_password),
                         metadata_store_factory=partial(solr_factory, solr_host_and_port))
-    if await run_health_checks(
-            [CassandraStore(cassandra_contact_points, cassandra_port),
-             SolrStore(solr_host_and_port),
-             consumer]):
+    if await run_health_checks([CassandraStore(cassandra_contact_points,
+                                               cassandra_port,
+                                               cassandra_username,
+                                               cassandra_password),
+                                SolrStore(solr_host_and_port),
+                                consumer]):
         async with consumer:
             logger.info("All external dependencies have passed the health checks. Now listening to message queue.")
             await consumer.start_consuming()
diff --git a/granule_ingester/granule_ingester/writers/CassandraStore.py b/granule_ingester/granule_ingester/writers/CassandraStore.py
index 7a9f146..cb5232b 100644
--- a/granule_ingester/granule_ingester/writers/CassandraStore.py
+++ b/granule_ingester/granule_ingester/writers/CassandraStore.py
@@ -18,11 +18,14 @@ import asyncio
 import logging
 import uuid
 
-from cassandra.cluster import Cluster, Session
+from cassandra.auth import PlainTextAuthProvider
+from cassandra.cluster import Cluster, Session, NoHostAvailable
 from cassandra.cqlengine import columns
 from cassandra.cqlengine.models import Model
+from cassandra.policies import RetryPolicy, ConstantReconnectionPolicy
 from nexusproto.DataTile_pb2 import NexusTile, TileData
 
+from granule_ingester.exceptions import CassandraFailedHealthCheckError, CassandraLostConnectionError
 from granule_ingester.writers.DataStore import DataStore
 
 logging.getLogger('cassandra').setLevel(logging.INFO)
@@ -37,8 +40,10 @@ class TileModel(Model):
 
 
 class CassandraStore(DataStore):
-    def __init__(self, contact_points=None, port=9042):
+    def __init__(self, contact_points=None, port=9042, username=None, password=None):
         self._contact_points = contact_points
+        self._username = username
+        self._password = password
         self._port = port
         self._session = None
 
@@ -47,12 +52,22 @@ class CassandraStore(DataStore):
             session = self._get_session()
             session.shutdown()
             return True
-        except:
-            logger.error("Cannot connect to Cassandra!")
-            return False
+        except Exception:
+            raise CassandraFailedHealthCheckError("Cannot connect to Cassandra!")
 
     def _get_session(self) -> Session:
-        cluster = Cluster(contact_points=self._contact_points, port=self._port)
+
+        if self._username and self._password:
+            auth_provider = PlainTextAuthProvider(username=self._username, password=self._password)
+        else:
+            auth_provider = None
+
+        cluster = Cluster(contact_points=self._contact_points,
+                          port=self._port,
+                          # load_balancing_policy=
+                          reconnection_policy=ConstantReconnectionPolicy(delay=5.0),
+                          default_retry_policy=RetryPolicy(),
+                          auth_provider=auth_provider)
         session = cluster.connect()
         session.set_keyspace('nexustiles')
         return session
@@ -65,10 +80,14 @@ class CassandraStore(DataStore):
             self._session.shutdown()
 
     async def save_data(self, tile: NexusTile) -> None:
-        tile_id = uuid.UUID(tile.summary.tile_id)
-        serialized_tile_data = TileData.SerializeToString(tile.tile)
-        prepared_query = self._session.prepare("INSERT INTO sea_surface_temp (tile_id, tile_blob) VALUES (?, ?)")
-        await type(self)._execute_query_async(self._session, prepared_query, [tile_id, bytearray(serialized_tile_data)])
+        try:
+            tile_id = uuid.UUID(tile.summary.tile_id)
+            serialized_tile_data = TileData.SerializeToString(tile.tile)
+            prepared_query = self._session.prepare("INSERT INTO sea_surface_temp (tile_id, tile_blob) VALUES (?, ?)")
+            await self._execute_query_async(self._session, prepared_query,
+                                            [tile_id, bytearray(serialized_tile_data)])
+        except NoHostAvailable:
+            raise CassandraLostConnectionError(f"Lost connection to Cassandra, and cannot save tiles.")
 
     @staticmethod
     async def _execute_query_async(session: Session, query, parameters=None):

[incubator-sdap-ingester] 26/33: SDAP-300: Fix bug that prevented collection manager from seeing files in a directory when the path had no glob-style wildcard character. Also add back support for scanning dirs recursively. (#27)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 7448280e070b28efbf99ec49b3c2954aac052c6e
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Jan 5 10:05:04 2021 -0800

    SDAP-300: Fix bug that prevented collection manager from seeing files in a directory when the path had no glob-style wildcard character. Also add back support for scanning dirs recursively. (#27)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 .../collection_manager/entities/Collection.py              |  7 ++++---
 .../collection_manager/services/CollectionWatcher.py       | 14 +++++++++++---
 2 files changed, 15 insertions(+), 6 deletions(-)

diff --git a/collection_manager/collection_manager/entities/Collection.py b/collection_manager/collection_manager/entities/Collection.py
index 7a45b66..389e135 100644
--- a/collection_manager/collection_manager/entities/Collection.py
+++ b/collection_manager/collection_manager/entities/Collection.py
@@ -1,11 +1,12 @@
 import os
-from urllib.parse import urlparse
+import pathlib
 from dataclasses import dataclass
 from datetime import datetime
+from enum import Enum
 from fnmatch import fnmatch
 from glob import glob
 from typing import List, Optional
-from enum import Enum
+from urllib.parse import urlparse
 
 from collection_manager.entities.exceptions import MissingValueCollectionError
 
@@ -68,6 +69,6 @@ class Collection:
                 raise IsADirectoryError()
 
             if os.path.isdir(self.path):
-                return os.path.dirname(file_path) == self.path
+                return pathlib.Path(self.path) in pathlib.Path(file_path).parents
             else:
                 return fnmatch(file_path, self.path)
diff --git a/collection_manager/collection_manager/services/CollectionWatcher.py b/collection_manager/collection_manager/services/CollectionWatcher.py
index b1aaf4e..68b013a 100644
--- a/collection_manager/collection_manager/services/CollectionWatcher.py
+++ b/collection_manager/collection_manager/services/CollectionWatcher.py
@@ -116,11 +116,16 @@ class CollectionWatcher:
         logger.info(f"Scanning files for {len(collections)} collections...")
         start = time.perf_counter()
         for collection in collections:
-            for granule_path in glob(collection.path, recursive=True):
+            for granule_path in self._get_files_at_path(collection.path):
                 modified_time = int(os.path.getmtime(granule_path))
                 await self._granule_updated_callback(granule_path, modified_time, collection)
         logger.info(f"Finished scanning files in {time.perf_counter() - start} seconds.")
 
+    def _get_files_at_path(self, path: str) -> List[str]:
+        if os.path.isfile(path):
+            return [path]
+        return [f for f in glob(path + '/**', recursive=True) if os.path.isfile(f)]
+
     async def _reload_and_reschedule(self):
         try:
             updated_collections = self._get_updated_collections()
@@ -191,11 +196,14 @@ class _GranuleEventHandler(FileSystemEventHandler):
 
     def on_created(self, event):
         super().on_created(event)
-        self._handle_event(event)
+        if isinstance(event, S3Event) or not event.is_directory:
+            self._handle_event(event)
 
     def on_modified(self, event):
         super().on_modified(event)
-        self._handle_event(event)
+
+        if isinstance(event, S3Event) or not event.is_directory:
+            self._handle_event(event)
 
     def _handle_event(self, event):
         path = event.src_path

[incubator-sdap-ingester] 27/33: KelvinToCelsius converter can now parse 'units' or 'Units' field and exits gracefully if none exists (#28)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 677fb5766df47e0bdad0a9c084830d4bf99b4333
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Tue Jan 5 14:03:47 2021 -0800

    KelvinToCelsius converter can now parse 'units' or 'Units' field and exits gracefully if none exists (#28)
---
 collection_manager/collection_manager/services/S3Observer.py   |  2 --
 .../granule_ingester/processors/kelvintocelsius.py             | 10 +++++++++-
 2 files changed, 9 insertions(+), 3 deletions(-)

diff --git a/collection_manager/collection_manager/services/S3Observer.py b/collection_manager/collection_manager/services/S3Observer.py
index 87458a9..6d87d91 100644
--- a/collection_manager/collection_manager/services/S3Observer.py
+++ b/collection_manager/collection_manager/services/S3Observer.py
@@ -109,8 +109,6 @@ class S3Observer:
         end = time.perf_counter()
         duration = end - start
 
-        print(f"Retrieved {len(new_cache)} objects in {duration}")
-
         return new_cache
 
     def _get_object_key(full_path: str):
diff --git a/granule_ingester/granule_ingester/processors/kelvintocelsius.py b/granule_ingester/granule_ingester/processors/kelvintocelsius.py
index 9ad4f49..c0a9285 100644
--- a/granule_ingester/granule_ingester/processors/kelvintocelsius.py
+++ b/granule_ingester/granule_ingester/processors/kelvintocelsius.py
@@ -27,7 +27,15 @@ class KelvinToCelsius(TileProcessor):
         if 'dataset' in kwargs:
             ds = kwargs['dataset']
             variable_name = tile.summary.data_var_name
-            variable_unit = ds.variables[variable_name].attrs['units']
+            if 'units' in ds.variables[variable_name].attrs:
+                variable_unit = ds.variables[variable_name].attrs['units']
+            elif 'Units' in ds.variables[variable_name].attrs:
+                variable_unit = ds.variables[variable_name].attrs['Units']
+            elif 'UNITS' in ds.variables[variable_name].attrs:
+                variable_unit = ds.variables[variable_name].attrs['UNITS']
+            else:
+                return tile
+
             if any([unit in variable_unit.lower() for unit in kelvins]):
                 var_data = from_shaped_array(the_tile_data.variable_data) - 273.15
                 the_tile_data.variable_data.CopyFrom(to_shaped_array(var_data))

[incubator-sdap-ingester] 32/33: restore files from lfs

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit 54b8eec2407f297364585abca81ecb343ae24d3b
Author: thomas loubrieu <th...@jpl.nasa.gov>
AuthorDate: Thu Apr 8 10:56:37 2021 -0700

    restore files from lfs
---
 ...4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc | Bin 132 -> 1057327 bytes
 granule_ingester/tests/granules/OBP_2017_01.nc     | Bin 132 -> 2110135 bytes
 granule_ingester/tests/granules/OBP_native_grid.nc | Bin 132 -> 1285094 bytes
 .../SMAP_L2B_SSS_04892_20160101T005507_R13080.h5   | Bin 133 -> 18672352 bytes
 granule_ingester/tests/granules/THETA_199201.nc    | Bin 132 -> 4255957 bytes
 granule_ingester/tests/granules/empty_mur.nc4      | Bin 130 -> 60937 bytes
 .../tests/granules/not_empty_ascatb.nc4            | Bin 130 -> 78036 bytes
 .../tests/granules/not_empty_avhrr.nc4             | Bin 130 -> 49511 bytes
 granule_ingester/tests/granules/not_empty_ccmp.nc  | Bin 131 -> 206870 bytes
 9 files changed, 0 insertions(+), 0 deletions(-)

diff --git a/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc b/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc
index 6996466..4935c81 100644
Binary files a/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc and b/granule_ingester/tests/granules/20050101120000-NCEI-L4_GHRSST-SSTblend-AVHRR_OI-GLOB-v02.0-fv02.0.nc differ
diff --git a/granule_ingester/tests/granules/OBP_2017_01.nc b/granule_ingester/tests/granules/OBP_2017_01.nc
index 8401b34..8c9b584 100644
Binary files a/granule_ingester/tests/granules/OBP_2017_01.nc and b/granule_ingester/tests/granules/OBP_2017_01.nc differ
diff --git a/granule_ingester/tests/granules/OBP_native_grid.nc b/granule_ingester/tests/granules/OBP_native_grid.nc
index f4dbe6a..addb8a0 100755
Binary files a/granule_ingester/tests/granules/OBP_native_grid.nc and b/granule_ingester/tests/granules/OBP_native_grid.nc differ
diff --git a/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 b/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5
index 8a5d950..11815dd 100644
Binary files a/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 and b/granule_ingester/tests/granules/SMAP_L2B_SSS_04892_20160101T005507_R13080.h5 differ
diff --git a/granule_ingester/tests/granules/THETA_199201.nc b/granule_ingester/tests/granules/THETA_199201.nc
index d650b72..ad92a61 100644
Binary files a/granule_ingester/tests/granules/THETA_199201.nc and b/granule_ingester/tests/granules/THETA_199201.nc differ
diff --git a/granule_ingester/tests/granules/empty_mur.nc4 b/granule_ingester/tests/granules/empty_mur.nc4
index 96c4980..f65c808 100644
Binary files a/granule_ingester/tests/granules/empty_mur.nc4 and b/granule_ingester/tests/granules/empty_mur.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_ascatb.nc4 b/granule_ingester/tests/granules/not_empty_ascatb.nc4
index 21b48cc..d8ef90b 100644
Binary files a/granule_ingester/tests/granules/not_empty_ascatb.nc4 and b/granule_ingester/tests/granules/not_empty_ascatb.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_avhrr.nc4 b/granule_ingester/tests/granules/not_empty_avhrr.nc4
index f25f916..af24071 100644
Binary files a/granule_ingester/tests/granules/not_empty_avhrr.nc4 and b/granule_ingester/tests/granules/not_empty_avhrr.nc4 differ
diff --git a/granule_ingester/tests/granules/not_empty_ccmp.nc b/granule_ingester/tests/granules/not_empty_ccmp.nc
index 5a215d3..b7b491d 100644
Binary files a/granule_ingester/tests/granules/not_empty_ccmp.nc and b/granule_ingester/tests/granules/not_empty_ccmp.nc differ

[incubator-sdap-ingester] 15/33: SDAP-277: Improved error handling in Granule Ingester (#15)

Posted by tl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tloubrieu pushed a commit to branch ascending_latitudes
in repository https://gitbox.apache.org/repos/asf/incubator-sdap-ingester.git

commit caa296af7976c046996f22677174574247630ca3
Author: Eamon Ford <ea...@gmail.com>
AuthorDate: Wed Aug 5 19:28:07 2020 -0700

    SDAP-277: Improved error handling in Granule Ingester (#15)
    
    Co-authored-by: Eamon Ford <ea...@jpl.nasa.gov>
---
 granule_ingester/conda-requirements.txt            |   2 +-
 .../granule_ingester/consumer/Consumer.py          |  40 +++++--
 .../granule_ingester/pipeline/Pipeline.py          | 132 +++++++++++++--------
 .../reading_processors/TileReadingProcessor.py     |  20 ++--
 granule_ingester/tests/pipeline/test_Pipeline.py   |   9 +-
 5 files changed, 130 insertions(+), 73 deletions(-)

diff --git a/granule_ingester/conda-requirements.txt b/granule_ingester/conda-requirements.txt
index b2af149..fafd6f3 100644
--- a/granule_ingester/conda-requirements.txt
+++ b/granule_ingester/conda-requirements.txt
@@ -6,5 +6,5 @@ xarray
 pyyaml==5.3.1
 requests==2.23.0
 aiohttp==3.6.2
-aio-pika
+aio-pika==6.6.1
 tenacity
diff --git a/granule_ingester/granule_ingester/consumer/Consumer.py b/granule_ingester/granule_ingester/consumer/Consumer.py
index 5df51fe..6c72837 100644
--- a/granule_ingester/granule_ingester/consumer/Consumer.py
+++ b/granule_ingester/granule_ingester/consumer/Consumer.py
@@ -17,6 +17,8 @@ import logging
 
 import aio_pika
 
+from granule_ingester.exceptions import PipelineBuildingError, PipelineRunningError, RabbitMQLostConnectionError, \
+    RabbitMQFailedHealthCheckError, LostConnectionError
 from granule_ingester.healthcheck import HealthCheck
 from granule_ingester.pipeline import Pipeline
 
@@ -39,7 +41,7 @@ class Consumer(HealthCheck):
         self._connection_string = "amqp://{username}:{password}@{host}/".format(username=rabbitmq_username,
                                                                                 password=rabbitmq_password,
                                                                                 host=rabbitmq_host)
-        self._connection = None
+        self._connection: aio_pika.Connection = None
 
     async def health_check(self) -> bool:
         try:
@@ -47,10 +49,10 @@ class Consumer(HealthCheck):
             await connection.close()
             return True
         except Exception:
-            logger.error("Cannot connect to RabbitMQ! Connection string was {}".format(self._connection_string))
-            return False
+            raise RabbitMQFailedHealthCheckError(f"Cannot connect to RabbitMQ! "
+                                                 f"Connection string was {self._connection_string}")
 
-    async def _get_connection(self):
+    async def _get_connection(self) -> aio_pika.Connection:
         return await aio_pika.connect_robust(self._connection_string)
 
     async def __aenter__(self):
@@ -75,19 +77,37 @@ class Consumer(HealthCheck):
                                             metadata_store_factory=metadata_store_factory,
                                             max_concurrency=pipeline_max_concurrency)
             await pipeline.run()
-            message.ack()
+            await message.ack()
+        except PipelineBuildingError as e:
+            await message.reject()
+            logger.exception(f"Failed to build the granule-processing pipeline. This message will be dropped "
+                             f"from RabbitMQ. The exception was:\n{e}")
+        except PipelineRunningError as e:
+            await message.reject()
+            logger.exception(f"Processing the granule failed. It will not be retried. The exception was:\n{e}")
+        except LostConnectionError:
+            # Let main() handle this
+            raise
         except Exception as e:
-            message.reject(requeue=True)
-            logger.error("Processing message failed. Message will be re-queued. The exception was:\n{}".format(e))
+            await message.reject(requeue=True)
+            logger.exception(f"Processing message failed. Message will be re-queued. The exception was:\n{e}")
 
     async def start_consuming(self, pipeline_max_concurrency=16):
         channel = await self._connection.channel()
         await channel.set_qos(prefetch_count=1)
         queue = await channel.declare_queue(self._rabbitmq_queue, durable=True)
-
-        async with queue.iterator() as queue_iter:
-            async for message in queue_iter:
+        queue_iter = queue.iterator()
+        async for message in queue_iter:
+            try:
                 await self._received_message(message,
                                              self._data_store_factory,
                                              self._metadata_store_factory,
                                              pipeline_max_concurrency)
+            except aio_pika.exceptions.MessageProcessError:
+                # Do not try to close() the queue iterator! If we get here, that means the RabbitMQ
+                # connection has died, and attempting to close the queue will only raise another exception.
+                raise RabbitMQLostConnectionError("Lost connection to RabbitMQ while processing a granule.")
+            except Exception as e:
+                await queue_iter.close()
+                await channel.close()
+                raise e
diff --git a/granule_ingester/granule_ingester/pipeline/Pipeline.py b/granule_ingester/granule_ingester/pipeline/Pipeline.py
index e1e53bf..dabca81 100644
--- a/granule_ingester/granule_ingester/pipeline/Pipeline.py
+++ b/granule_ingester/granule_ingester/pipeline/Pipeline.py
@@ -13,38 +13,46 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-
 import logging
-import os
+import pickle
 import time
+from multiprocessing import Manager
 from typing import List
 
 import xarray as xr
 import yaml
 
-import aiomultiprocess
+from aiomultiprocess import Pool
+from aiomultiprocess.types import ProxyException
+from granule_ingester.exceptions import PipelineBuildingError
 from granule_ingester.granule_loaders import GranuleLoader
-from granule_ingester.pipeline.Modules import modules as processor_module_mappings
+from granule_ingester.pipeline.Modules import \
+    modules as processor_module_mappings
 from granule_ingester.processors.TileProcessor import TileProcessor
 from granule_ingester.slicers import TileSlicer
 from granule_ingester.writers import DataStore, MetadataStore
 from nexusproto import DataTile_pb2 as nexusproto
+from tblib import pickling_support
 
 logger = logging.getLogger(__name__)
 
-MAX_QUEUE_SIZE = 2 ** 15 - 1
+# The aiomultiprocessing library has a bug where it never closes out the pool if there are more than a certain
+# number of items to process. The exact number is unknown, but 2**8-1 is safe.
+MAX_CHUNK_SIZE = 2 ** 8 - 1
 
 _worker_data_store: DataStore = None
 _worker_metadata_store: MetadataStore = None
 _worker_processor_list: List[TileProcessor] = None
 _worker_dataset = None
+_shared_memory = None
 
 
-def _init_worker(processor_list, dataset, data_store_factory, metadata_store_factory):
+def _init_worker(processor_list, dataset, data_store_factory, metadata_store_factory, shared_memory):
     global _worker_data_store
     global _worker_metadata_store
     global _worker_processor_list
     global _worker_dataset
+    global _shared_memory
 
     # _worker_data_store and _worker_metadata_store open multiple TCP sockets from each worker process;
     # however, these sockets will be automatically closed by the OS once the worker processes die so no need to worry.
@@ -52,19 +60,21 @@ def _init_worker(processor_list, dataset, data_store_factory, metadata_store_fac
     _worker_metadata_store = metadata_store_factory()
     _worker_processor_list = processor_list
     _worker_dataset = dataset
+    _shared_memory = shared_memory
 
 
 async def _process_tile_in_worker(serialized_input_tile: str):
-    global _worker_data_store
-    global _worker_metadata_store
-    global _worker_processor_list
-    global _worker_dataset
+    try:
+        input_tile = nexusproto.NexusTile.FromString(serialized_input_tile)
+        processed_tile = _recurse(_worker_processor_list, _worker_dataset, input_tile)
 
-    input_tile = nexusproto.NexusTile.FromString(serialized_input_tile)
-    processed_tile = _recurse(_worker_processor_list, _worker_dataset, input_tile)
-    if processed_tile:
-        await _worker_data_store.save_data(processed_tile)
-        await _worker_metadata_store.save_metadata(processed_tile)
+        if processed_tile:
+            await _worker_data_store.save_data(processed_tile)
+            await _worker_metadata_store.save_metadata(processed_tile)
+    except Exception as e:
+        pickling_support.install(e)
+        _shared_memory.error = pickle.dumps(e)
+        raise
 
 
 def _recurse(processor_list: List[TileProcessor],
@@ -91,25 +101,34 @@ class Pipeline:
         self._metadata_store_factory = metadata_store_factory
         self._max_concurrency = max_concurrency
 
-    @classmethod
-    def from_string(cls, config_str: str, data_store_factory, metadata_store_factory, max_concurrency: int = 16):
-        config = yaml.load(config_str, yaml.FullLoader)
-        return cls._build_pipeline(config,
-                                   data_store_factory,
-                                   metadata_store_factory,
-                                   processor_module_mappings,
-                                   max_concurrency)
+        # Create a SyncManager so that we can to communicate exceptions from the
+        # worker processes back to the main process.
+        self._manager = Manager()
+
+    def __del__(self):
+        self._manager.shutdown()
 
     @classmethod
-    def from_file(cls, config_path: str, data_store_factory, metadata_store_factory, max_concurrency: int = 16):
-        with open(config_path) as config_file:
-            config = yaml.load(config_file, yaml.FullLoader)
+    def from_string(cls, config_str: str, data_store_factory, metadata_store_factory, max_concurrency: int = 16):
+        try:
+            config = yaml.load(config_str, yaml.FullLoader)
+            cls._validate_config(config)
             return cls._build_pipeline(config,
                                        data_store_factory,
                                        metadata_store_factory,
                                        processor_module_mappings,
                                        max_concurrency)
 
+        except yaml.scanner.ScannerError:
+            raise PipelineBuildingError("Cannot build pipeline because of a syntax error in the YAML.")
+
+    # TODO: this method should validate the config against an actual schema definition
+    @staticmethod
+    def _validate_config(config: dict):
+        if type(config) is not dict:
+            raise PipelineBuildingError("Cannot build pipeline; the pipeline configuration that " +
+                                        "was received is not valid YAML.")
+
     @classmethod
     def _build_pipeline(cls,
                         config: dict,
@@ -117,17 +136,27 @@ class Pipeline:
                         metadata_store_factory,
                         module_mappings: dict,
                         max_concurrency: int):
-        granule_loader = GranuleLoader(**config['granule'])
-
-        slicer_config = config['slicer']
-        slicer = cls._parse_module(slicer_config, module_mappings)
-
-        tile_processors = []
-        for processor_config in config['processors']:
-            module = cls._parse_module(processor_config, module_mappings)
-            tile_processors.append(module)
-
-        return cls(granule_loader, slicer, data_store_factory, metadata_store_factory, tile_processors, max_concurrency)
+        try:
+            granule_loader = GranuleLoader(**config['granule'])
+
+            slicer_config = config['slicer']
+            slicer = cls._parse_module(slicer_config, module_mappings)
+
+            tile_processors = []
+            for processor_config in config['processors']:
+                module = cls._parse_module(processor_config, module_mappings)
+                tile_processors.append(module)
+
+            return cls(granule_loader,
+                       slicer,
+                       data_store_factory,
+                       metadata_store_factory,
+                       tile_processors,
+                       max_concurrency)
+        except KeyError as e:
+            raise PipelineBuildingError(f"Cannot build pipeline because {e} is missing from the YAML.")
+        except Exception:
+            raise PipelineBuildingError("Cannot build pipeline.")
 
     @classmethod
     def _parse_module(cls, module_config: dict, module_mappings: dict):
@@ -144,23 +173,32 @@ class Pipeline:
     async def run(self):
         async with self._granule_loader as (dataset, granule_name):
             start = time.perf_counter()
-            async with aiomultiprocess.Pool(initializer=_init_worker,
-                                            initargs=(self._tile_processors,
-                                                      dataset,
-                                                      self._data_store_factory,
-                                                      self._metadata_store_factory),
-                                            maxtasksperchild=self._max_concurrency,
-                                            childconcurrency=self._max_concurrency) as pool:
+
+            shared_memory = self._manager.Namespace()
+            async with Pool(initializer=_init_worker,
+                            initargs=(self._tile_processors,
+                                      dataset,
+                                      self._data_store_factory,
+                                      self._metadata_store_factory,
+                                      shared_memory),
+                            maxtasksperchild=self._max_concurrency,
+                            childconcurrency=self._max_concurrency) as pool:
                 serialized_tiles = [nexusproto.NexusTile.SerializeToString(tile) for tile in
                                     self._slicer.generate_tiles(dataset, granule_name)]
                 # aiomultiprocess is built on top of the stdlib multiprocessing library, which has the limitation that
                 # a queue can't have more than 2**15-1 tasks. So, we have to batch it.
-                for chunk in type(self)._chunk_list(serialized_tiles, MAX_QUEUE_SIZE):
-                    await pool.map(_process_tile_in_worker, chunk)
+                for chunk in self._chunk_list(serialized_tiles, MAX_CHUNK_SIZE):
+                    try:
+                        await pool.map(_process_tile_in_worker, chunk)
+                    except ProxyException:
+                        pool.terminate()
+                        # Give the shared memory manager some time to write the exception
+                        # await asyncio.sleep(1)
+                        raise pickle.loads(shared_memory.error)
 
         end = time.perf_counter()
         logger.info("Pipeline finished in {} seconds".format(end - start))
 
     @staticmethod
-    def _chunk_list(items, chunk_size):
+    def _chunk_list(items, chunk_size: int):
         return [items[i:i + chunk_size] for i in range(0, len(items), chunk_size)]
diff --git a/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py b/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
index 14a44f5..8b69ad2 100644
--- a/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
+++ b/granule_ingester/granule_ingester/processors/reading_processors/TileReadingProcessor.py
@@ -21,6 +21,7 @@ import numpy as np
 import xarray as xr
 from nexusproto import DataTile_pb2 as nexusproto
 
+from granule_ingester.exceptions import TileProcessingError
 from granule_ingester.processors.TileProcessor import TileProcessor
 
 
@@ -31,20 +32,17 @@ class TileReadingProcessor(TileProcessor, ABC):
         self.latitude = latitude
         self.longitude = longitude
 
-        # Common optional properties
-        self.temp_dir = None
-        self.metadata = None
-        # self.temp_dir = self.environ['TEMP_DIR']
-        # self.metadata = self.environ['META']
-
     def process(self, tile, dataset: xr.Dataset, *args, **kwargs):
-        dimensions_to_slices = type(self)._convert_spec_to_slices(tile.summary.section_spec)
+        try:
+            dimensions_to_slices = self._convert_spec_to_slices(tile.summary.section_spec)
 
-        output_tile = nexusproto.NexusTile()
-        output_tile.CopyFrom(tile)
-        output_tile.summary.data_var_name = self.variable_to_read
+            output_tile = nexusproto.NexusTile()
+            output_tile.CopyFrom(tile)
+            output_tile.summary.data_var_name = self.variable_to_read
 
-        return self._generate_tile(dataset, dimensions_to_slices, output_tile)
+            return self._generate_tile(dataset, dimensions_to_slices, output_tile)
+        except Exception:
+            raise TileProcessingError("Could not generate tiles from the granule.")
 
     @abstractmethod
     def _generate_tile(self, dataset: xr.Dataset, dimensions_to_slices: Dict[str, slice], tile):
diff --git a/granule_ingester/tests/pipeline/test_Pipeline.py b/granule_ingester/tests/pipeline/test_Pipeline.py
index c18bf8b..34e66c6 100644
--- a/granule_ingester/tests/pipeline/test_Pipeline.py
+++ b/granule_ingester/tests/pipeline/test_Pipeline.py
@@ -29,10 +29,11 @@ class TestPipeline(unittest.TestCase):
                 pass
 
         relative_path = "../config_files/ingestion_config_testfile.yaml"
-        file_path = os.path.join(os.path.dirname(__file__), relative_path)
-        pipeline = Pipeline.from_file(config_path=str(file_path),
-                                      data_store_factory=MockDataStore,
-                                      metadata_store_factory=MockMetadataStore)
+        with open(os.path.join(os.path.dirname(__file__), relative_path)) as file:
+            yaml_str = file.read()
+        pipeline = Pipeline.from_string(config_str=yaml_str,
+                                        data_store_factory=MockDataStore,
+                                        metadata_store_factory=MockMetadataStore)
 
         self.assertEqual(pipeline._data_store_factory, MockDataStore)
         self.assertEqual(pipeline._metadata_store_factory, MockMetadataStore)