You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@sdap.apache.org by GitBox <gi...@apache.org> on 2020/06/26 01:14:50 UTC

[GitHub] [incubator-sdap-nexus] tloubrieu-jpl opened a new pull request #104: Sdap 252

tloubrieu-jpl opened a new pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104


   This pull request removes the singleton pattern for Nexus Calc Handers.
   
   This also simplifies the hierarchy of handlers and distribute objects in their own file (for code related to the pull request).
   
   The updates have been tested on the following algorithms but not on the others:
                 'capabilities'
                 'heartbeat'
                 'delay'
                 'makeerror'
                 'timeAvgMapSpark'
                 'timeSeriesSpark'
                 'varianceSpark'
                 'latitudeTimeHofMoellerSpark'
                 'longitudeTimeHofMoellerSpark'
   
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451859321



##########
File path: analysis/webservice/webmodel/NexusRequestObject.py
##########
@@ -0,0 +1,227 @@
+import logging
+import re
+from datetime import datetime
+from decimal import Decimal
+
+from pytz import UTC
+from shapely.geometry import Polygon
+from webservice.webmodel.RequestParameters import RequestParameters
+from webservice.webmodel.StatsComputeOptions import StatsComputeOptions
+
+
+class NexusRequestObject(StatsComputeOptions):
+    shortNamePattern = re.compile("^[a-zA-Z0-9_\-,\.]+$")
+    floatingPointPattern = re.compile('[+-]?(\d+(\.\d*)?|\.\d+)([eE][+-]?\d+)?')
+
+    def __init__(self, reqHandler):
+        self.__log = logging.getLogger(__name__)
+        if reqHandler is None:
+            raise Exception("Request handler cannot be null")
+        self.requestHandler = reqHandler
+        StatsComputeOptions.__init__(self)
+
+    def get_argument(self, name, default=None):
+        return self.requestHandler.get_argument(name, default=default)
+
+    def get_list_int_arg(self, name, default=None):
+        arg = self.get_argument(name, default=default)
+        return arg.split(',')
+
+    def __validate_is_shortname(self, v):
+        if v is None or len(v) == 0:
+            return False
+        return self.shortNamePattern.match(v) is not None
+
+    def __validate_is_number(self, v):
+        if v is None or (type(v) == str and len(v) == 0):
+            return False
+        elif type(v) == int or type(v) == float:
+            return True
+        else:
+            return self.floatingPointPattern.match(v) is not None
+
+    def get_float_arg(self, name, default=0.0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return float(arg)
+        else:
+            return default
+
+    def get_decimal_arg(self, name, default=0.0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return Decimal(arg)
+        else:
+            if default is None:
+                return None
+            return Decimal(default)
+
+    def get_int_arg(self, name, default=0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return int(arg)
+        else:
+            return default
+
+    def get_boolean_arg(self, name, default=False):
+        arg = self.get_argument(name, "false" if not default else "true")
+        return arg is not None and arg in ['true', '1', 't', 'y', 'yes', 'True', 'T', 'Y',
+                                           'Yes', True]
+
+    def get_datetime_arg(self, name, default=None):
+        time_str = self.get_argument(name, default=default)
+        if time_str == default:
+            return default
+        try:
+            dt = datetime.strptime(time_str, "%Y-%m-%dT%H:%M:%SZ").replace(tzinfo=UTC)
+        except ValueError:
+            dt = datetime.utcfromtimestamp(int(time_str)).replace(tzinfo=UTC)
+        return dt
+
+    def get_apply_seasonal_cycle_filter(self, default=True):
+        return self.get_boolean_arg(RequestParameters.SEASONAL_CYCLE_FILTER, default=default)
+
+    def get_max_lat(self, default=Decimal(90)):
+        return self.get_decimal_arg("maxLat", default)
+
+    def get_min_lat(self, default=Decimal(-90)):
+        return self.get_decimal_arg("minLat", default)
+
+    def get_max_lon(self, default=Decimal(180)):
+        return self.get_decimal_arg("maxLon", default)
+
+    def get_min_lon(self, default=Decimal(-180)):
+        return self.get_decimal_arg("minLon", default)
+
+    # added to fit the simplified version of TimeAvgMapSpark parse_argumemt
+    def get_bounding_box(self):
+
+        b = self.get_argument("b", '')
+        if b:
+            min_lon, min_lat, max_lon, max_lat = [float(e) for e in b.split(",")]
+        else:
+            max_lat = self.get_argument("maxLat", 90)
+            max_lat = Decimal(max_lat) if self.__validate_is_number(max_lat) else 90
+
+            min_lat = self.get_argument("minLat", -90)
+            min_lat = Decimal(min_lat) if self.__validate_is_number(min_lat) else -90
+
+            max_lon = self.get_argument("maxLon", 180)
+            max_lon = Decimal(max_lon) if self.__validate_is_number(max_lon) else 180
+
+            min_lon = self.get_argument("minLon", -90)
+            min_lon = Decimal(min_lon) if self.__validate_is_number(min_lon) else -90
+
+        return min_lon, min_lat, max_lon, max_lat
+
+
+    def get_bounding_polygon(self):
+        west, south, east, north = [float(b) for b in self.get_argument("b").split(",")]
+        polygon = Polygon([(west, south), (east, south), (east, north), (west, north), (west, south)])
+        return polygon
+
+    def get_dataset(self):
+        ds = self.get_argument(RequestParameters.DATASET, None)
+        if ds is not None and not self.__validate_is_shortname(ds):
+            raise Exception("Invalid shortname")
+        else:
+            return ds.split(",")
+
+    def get_metadata_filter(self):
+        return self.requestHandler.get_arguments(RequestParameters.METADATA_FILTER)
+
+    def get_environment(self):
+        env = self.get_argument(RequestParameters.ENVIRONMENT, None)
+        if env is None and "Origin" in self.requestHandler.request.headers:
+            origin = self.requestHandler.request.headers["Origin"]
+            if origin == "http://localhost:63342":
+                env = "DEV"
+            if origin == "https://sealevel.uat.earthdata.nasa.gov":
+                env = "UAT"
+            elif origin == "https://sealevel.sit.earthdata.nasa.gov":
+                env = "SIT"
+            elif origin == "https://sealevel.earthdata.nasa.gov":
+                env = "PROD"

Review comment:
       Yes that does look good. I need to revise this object because it is too much connected to a tornado request object which will change over time (for new user requests) and this is a point of failure for concurrent requests. Although it was not the problem we had initially with podaac test.
   
   I will do that in a separate ticket.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451854205



##########
File path: analysis/webservice/webmodel/NexusRequestObjectTornadoFree.py
##########
@@ -0,0 +1,108 @@
+import logging
+import re
+from datetime import datetime
+from decimal import Decimal
+
+from pytz import UTC
+from webservice.webmodel.RequestParameters import RequestParameters
+from webservice.webmodel.StatsComputeOptions import StatsComputeOptions
+
+
+class NexusRequestObjectTornadoFree(StatsComputeOptions):

Review comment:
       Yes that was what I had in mind.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451850842



##########
File path: analysis/webservice/algorithms_spark/ClimMapSpark.py
##########
@@ -22,22 +22,17 @@
 import numpy as np
 from nexustiles.nexustiles import NexusTileService
 
-from webservice.NexusHandler import nexus_handler, SparkHandler, DEFAULT_PARAMETERS_SPEC
+from webservice.NexusHandler import nexus_handler, DEFAULT_PARAMETERS_SPEC
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 
 @nexus_handler
-class ClimMapSparkHandlerImpl(SparkHandler):
+class ClimMapNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       I updated the typo.

##########
File path: analysis/webservice/algorithms_spark/DailyDifferenceAverageSpark.py
##########
@@ -37,7 +38,7 @@ def iso_time_to_epoch(str_time):
 
 
 @nexus_handler
-class DailyDifferenceAverageSparkImpl(SparkHandler):
+class DailyDifferenceAverageNexusImplSkark(NexusCalcSkarkHandler):

Review comment:
       updated




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] eamonford commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
eamonford commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r446363886



##########
File path: analysis/webservice/algorithms_spark/ClimMapSpark.py
##########
@@ -22,22 +22,17 @@
 import numpy as np
 from nexustiles.nexustiles import NexusTileService
 
-from webservice.NexusHandler import nexus_handler, SparkHandler, DEFAULT_PARAMETERS_SPEC
+from webservice.NexusHandler import nexus_handler, DEFAULT_PARAMETERS_SPEC
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 
 @nexus_handler
-class ClimMapSparkHandlerImpl(SparkHandler):
+class ClimMapNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       typo




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451853758



##########
File path: analysis/webservice/algorithms_spark/HofMoellerSpark.py
##########
@@ -100,7 +101,7 @@ def hofmoeller_stats(metrics_callback, tile_in_spark):
         return stats
 
 
-class BaseHoffMoellerHandlerImpl(SparkHandler):
+class BaseHoffMoellerSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       updated




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] echyam commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
echyam commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r452462570



##########
File path: analysis/webservice/webmodel/NexusRequestObject.py
##########
@@ -0,0 +1,227 @@
+import logging
+import re
+from datetime import datetime
+from decimal import Decimal
+
+from pytz import UTC
+from shapely.geometry import Polygon
+from webservice.webmodel.RequestParameters import RequestParameters
+from webservice.webmodel.StatsComputeOptions import StatsComputeOptions
+
+
+class NexusRequestObject(StatsComputeOptions):
+    shortNamePattern = re.compile("^[a-zA-Z0-9_\-,\.]+$")
+    floatingPointPattern = re.compile('[+-]?(\d+(\.\d*)?|\.\d+)([eE][+-]?\d+)?')
+
+    def __init__(self, reqHandler):
+        self.__log = logging.getLogger(__name__)
+        if reqHandler is None:
+            raise Exception("Request handler cannot be null")
+        self.requestHandler = reqHandler
+        StatsComputeOptions.__init__(self)
+
+    def get_argument(self, name, default=None):
+        return self.requestHandler.get_argument(name, default=default)
+
+    def get_list_int_arg(self, name, default=None):
+        arg = self.get_argument(name, default=default)
+        return arg.split(',')
+
+    def __validate_is_shortname(self, v):
+        if v is None or len(v) == 0:
+            return False
+        return self.shortNamePattern.match(v) is not None
+
+    def __validate_is_number(self, v):
+        if v is None or (type(v) == str and len(v) == 0):
+            return False
+        elif type(v) == int or type(v) == float:
+            return True
+        else:
+            return self.floatingPointPattern.match(v) is not None
+
+    def get_float_arg(self, name, default=0.0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return float(arg)
+        else:
+            return default
+
+    def get_decimal_arg(self, name, default=0.0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return Decimal(arg)
+        else:
+            if default is None:
+                return None
+            return Decimal(default)
+
+    def get_int_arg(self, name, default=0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return int(arg)
+        else:
+            return default
+
+    def get_boolean_arg(self, name, default=False):
+        arg = self.get_argument(name, "false" if not default else "true")
+        return arg is not None and arg in ['true', '1', 't', 'y', 'yes', 'True', 'T', 'Y',
+                                           'Yes', True]
+
+    def get_datetime_arg(self, name, default=None):
+        time_str = self.get_argument(name, default=default)
+        if time_str == default:
+            return default
+        try:
+            dt = datetime.strptime(time_str, "%Y-%m-%dT%H:%M:%SZ").replace(tzinfo=UTC)
+        except ValueError:
+            dt = datetime.utcfromtimestamp(int(time_str)).replace(tzinfo=UTC)
+        return dt
+
+    def get_apply_seasonal_cycle_filter(self, default=True):
+        return self.get_boolean_arg(RequestParameters.SEASONAL_CYCLE_FILTER, default=default)
+
+    def get_max_lat(self, default=Decimal(90)):
+        return self.get_decimal_arg("maxLat", default)
+
+    def get_min_lat(self, default=Decimal(-90)):
+        return self.get_decimal_arg("minLat", default)
+
+    def get_max_lon(self, default=Decimal(180)):
+        return self.get_decimal_arg("maxLon", default)
+
+    def get_min_lon(self, default=Decimal(-180)):
+        return self.get_decimal_arg("minLon", default)
+
+    # added to fit the simplified version of TimeAvgMapSpark parse_argumemt
+    def get_bounding_box(self):
+
+        b = self.get_argument("b", '')
+        if b:
+            min_lon, min_lat, max_lon, max_lat = [float(e) for e in b.split(",")]
+        else:
+            max_lat = self.get_argument("maxLat", 90)
+            max_lat = Decimal(max_lat) if self.__validate_is_number(max_lat) else 90
+
+            min_lat = self.get_argument("minLat", -90)
+            min_lat = Decimal(min_lat) if self.__validate_is_number(min_lat) else -90
+
+            max_lon = self.get_argument("maxLon", 180)
+            max_lon = Decimal(max_lon) if self.__validate_is_number(max_lon) else 180
+
+            min_lon = self.get_argument("minLon", -90)
+            min_lon = Decimal(min_lon) if self.__validate_is_number(min_lon) else -90
+
+        return min_lon, min_lat, max_lon, max_lat
+
+
+    def get_bounding_polygon(self):
+        west, south, east, north = [float(b) for b in self.get_argument("b").split(",")]
+        polygon = Polygon([(west, south), (east, south), (east, north), (west, north), (west, south)])
+        return polygon
+
+    def get_dataset(self):
+        ds = self.get_argument(RequestParameters.DATASET, None)
+        if ds is not None and not self.__validate_is_shortname(ds):
+            raise Exception("Invalid shortname")
+        else:
+            return ds.split(",")
+
+    def get_metadata_filter(self):
+        return self.requestHandler.get_arguments(RequestParameters.METADATA_FILTER)
+
+    def get_environment(self):
+        env = self.get_argument(RequestParameters.ENVIRONMENT, None)
+        if env is None and "Origin" in self.requestHandler.request.headers:
+            origin = self.requestHandler.request.headers["Origin"]
+            if origin == "http://localhost:63342":
+                env = "DEV"
+            if origin == "https://sealevel.uat.earthdata.nasa.gov":
+                env = "UAT"
+            elif origin == "https://sealevel.sit.earthdata.nasa.gov":
+                env = "SIT"
+            elif origin == "https://sealevel.earthdata.nasa.gov":
+                env = "PROD"

Review comment:
       ok, sounds good




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451853844



##########
File path: analysis/webservice/algorithms_spark/NexusCalcSparkHandler.py
##########
@@ -0,0 +1,359 @@
+import logging
+
+import numpy as np
+from netCDF4._netCDF4 import Dataset
+from webservice.algorithms.NexusCalcHandler import NexusCalcHandler
+from webservice.metrics import MetricsRecord, SparkAccumulatorMetricsField, NumberMetricsField
+from webservice.webmodel import NexusProcessingException
+
+
+class NexusCalcSkarkHandler(NexusCalcHandler):

Review comment:
       updated

##########
File path: analysis/webservice/algorithms_spark/TimeAvgMapSpark.py
##########
@@ -21,15 +21,18 @@
 import shapely.geometry
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 EPOCH = timezone('UTC').localize(datetime(1970, 1, 1))
 ISO_8601 = '%Y-%m-%dT%H:%M:%S%z'
 
 
 @nexus_handler
-class TimeAvgMapSparkHandlerImpl(SparkHandler):
+class TimeAvgMapNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       updated

##########
File path: analysis/webservice/algorithms_spark/TimeSeriesSpark.py
##########
@@ -42,7 +43,7 @@
 
 
 @nexus_handler
-class TimeSeriesHandlerImpl(SparkHandler):
+class TimeSeriesSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       updated

##########
File path: analysis/webservice/algorithms_spark/VarianceSpark.py
##########
@@ -23,15 +23,16 @@
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
 
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 EPOCH = timezone('UTC').localize(datetime(1970, 1, 1))
 ISO_8601 = '%Y-%m-%dT%H:%M:%S%z'
 
 
 @nexus_handler
-class VarianceSparkHandlerImpl(SparkHandler):
+class VarianceNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       updated




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451858049



##########
File path: analysis/webservice/webmodel/CustomEncoder.py
##########
@@ -0,0 +1,49 @@
+import json
+from datetime import datetime
+from decimal import Decimal
+
+import numpy as np
+
+
+class CustomEncoder(json.JSONEncoder):
+    def default(self, obj):
+        """If input object is an ndarray it will be converted into a dict
+        holding dtype, shape and the data, base64 encoded.
+        """
+        numpy_types = (
+            np.bool_,
+            # np.bytes_, -- python `bytes` class is not json serializable
+            # np.complex64,  -- python `complex` class is not json serializable
+            # np.complex128,  -- python `complex` class is not json serializable
+            # np.complex256,  -- python `complex` class is not json serializable
+            # np.datetime64,  -- python `datetime.datetime` class is not json serializable

Review comment:
       It is a piece of code I moved but I did not change the content. I don't even see where this class is used.
   
   Where do you think we should convert it to a string ? Generally speaking for code simplicity I would keep a datetime object in the code and only convert it from inputs or to outputs but there might  argument against that (size of the object ?).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl merged pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl merged pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#issuecomment-655798699


   I have created a follow on ticket SDAP-260 for the re-engineering for the NexusRequestObject which is linked to the tornado request object which is not thread safe.
   This will be an opportunity to remove hard coded values put there for sea level project.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] echyam commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
echyam commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r446427377



##########
File path: analysis/webservice/algorithms_spark/VarianceSpark.py
##########
@@ -23,15 +23,16 @@
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
 
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo lines 27, 35

##########
File path: analysis/webservice/algorithms_spark/TimeAvgMapSpark.py
##########
@@ -21,15 +21,18 @@
 import shapely.geometry
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo lines 25, 33

##########
File path: analysis/webservice/algorithms_spark/Matchup.py
##########
@@ -33,7 +33,8 @@
 from shapely.geometry import box
 from shapely.geos import ReadingError
 
-from webservice.NexusHandler import SparkHandler, nexus_handler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo lines 37, 54, 135

##########
File path: analysis/webservice/algorithms_spark/HofMoellerSpark.py
##########
@@ -27,7 +27,8 @@
 from matplotlib.ticker import FuncFormatter
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
-from webservice.NexusHandler import SparkHandler, nexus_handler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo lines 31, 104, 331, 335, 340, 382, 386, 391

##########
File path: analysis/webservice/webmodel/NexusRequestObject.py
##########
@@ -0,0 +1,227 @@
+import logging
+import re
+from datetime import datetime
+from decimal import Decimal
+
+from pytz import UTC
+from shapely.geometry import Polygon
+from webservice.webmodel.RequestParameters import RequestParameters
+from webservice.webmodel.StatsComputeOptions import StatsComputeOptions
+
+
+class NexusRequestObject(StatsComputeOptions):
+    shortNamePattern = re.compile("^[a-zA-Z0-9_\-,\.]+$")
+    floatingPointPattern = re.compile('[+-]?(\d+(\.\d*)?|\.\d+)([eE][+-]?\d+)?')
+
+    def __init__(self, reqHandler):
+        self.__log = logging.getLogger(__name__)
+        if reqHandler is None:
+            raise Exception("Request handler cannot be null")
+        self.requestHandler = reqHandler
+        StatsComputeOptions.__init__(self)
+
+    def get_argument(self, name, default=None):
+        return self.requestHandler.get_argument(name, default=default)
+
+    def get_list_int_arg(self, name, default=None):
+        arg = self.get_argument(name, default=default)
+        return arg.split(',')
+
+    def __validate_is_shortname(self, v):
+        if v is None or len(v) == 0:
+            return False
+        return self.shortNamePattern.match(v) is not None
+
+    def __validate_is_number(self, v):
+        if v is None or (type(v) == str and len(v) == 0):
+            return False
+        elif type(v) == int or type(v) == float:
+            return True
+        else:
+            return self.floatingPointPattern.match(v) is not None
+
+    def get_float_arg(self, name, default=0.0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return float(arg)
+        else:
+            return default
+
+    def get_decimal_arg(self, name, default=0.0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return Decimal(arg)
+        else:
+            if default is None:
+                return None
+            return Decimal(default)
+
+    def get_int_arg(self, name, default=0):
+        arg = self.get_argument(name, default)
+        if self.__validate_is_number(arg):
+            return int(arg)
+        else:
+            return default
+
+    def get_boolean_arg(self, name, default=False):
+        arg = self.get_argument(name, "false" if not default else "true")
+        return arg is not None and arg in ['true', '1', 't', 'y', 'yes', 'True', 'T', 'Y',
+                                           'Yes', True]
+
+    def get_datetime_arg(self, name, default=None):
+        time_str = self.get_argument(name, default=default)
+        if time_str == default:
+            return default
+        try:
+            dt = datetime.strptime(time_str, "%Y-%m-%dT%H:%M:%SZ").replace(tzinfo=UTC)
+        except ValueError:
+            dt = datetime.utcfromtimestamp(int(time_str)).replace(tzinfo=UTC)
+        return dt
+
+    def get_apply_seasonal_cycle_filter(self, default=True):
+        return self.get_boolean_arg(RequestParameters.SEASONAL_CYCLE_FILTER, default=default)
+
+    def get_max_lat(self, default=Decimal(90)):
+        return self.get_decimal_arg("maxLat", default)
+
+    def get_min_lat(self, default=Decimal(-90)):
+        return self.get_decimal_arg("minLat", default)
+
+    def get_max_lon(self, default=Decimal(180)):
+        return self.get_decimal_arg("maxLon", default)
+
+    def get_min_lon(self, default=Decimal(-180)):
+        return self.get_decimal_arg("minLon", default)
+
+    # added to fit the simplified version of TimeAvgMapSpark parse_argumemt
+    def get_bounding_box(self):
+
+        b = self.get_argument("b", '')
+        if b:
+            min_lon, min_lat, max_lon, max_lat = [float(e) for e in b.split(",")]
+        else:
+            max_lat = self.get_argument("maxLat", 90)
+            max_lat = Decimal(max_lat) if self.__validate_is_number(max_lat) else 90
+
+            min_lat = self.get_argument("minLat", -90)
+            min_lat = Decimal(min_lat) if self.__validate_is_number(min_lat) else -90
+
+            max_lon = self.get_argument("maxLon", 180)
+            max_lon = Decimal(max_lon) if self.__validate_is_number(max_lon) else 180
+
+            min_lon = self.get_argument("minLon", -90)
+            min_lon = Decimal(min_lon) if self.__validate_is_number(min_lon) else -90
+
+        return min_lon, min_lat, max_lon, max_lat
+
+
+    def get_bounding_polygon(self):
+        west, south, east, north = [float(b) for b in self.get_argument("b").split(",")]
+        polygon = Polygon([(west, south), (east, south), (east, north), (west, north), (west, south)])
+        return polygon
+
+    def get_dataset(self):
+        ds = self.get_argument(RequestParameters.DATASET, None)
+        if ds is not None and not self.__validate_is_shortname(ds):
+            raise Exception("Invalid shortname")
+        else:
+            return ds.split(",")
+
+    def get_metadata_filter(self):
+        return self.requestHandler.get_arguments(RequestParameters.METADATA_FILTER)
+
+    def get_environment(self):
+        env = self.get_argument(RequestParameters.ENVIRONMENT, None)
+        if env is None and "Origin" in self.requestHandler.request.headers:
+            origin = self.requestHandler.request.headers["Origin"]
+            if origin == "http://localhost:63342":
+                env = "DEV"
+            if origin == "https://sealevel.uat.earthdata.nasa.gov":
+                env = "UAT"
+            elif origin == "https://sealevel.sit.earthdata.nasa.gov":
+                env = "SIT"
+            elif origin == "https://sealevel.earthdata.nasa.gov":
+                env = "PROD"

Review comment:
       these URLs are specific to SLCP

##########
File path: analysis/webservice/webmodel/CustomEncoder.py
##########
@@ -0,0 +1,49 @@
+import json
+from datetime import datetime
+from decimal import Decimal
+
+import numpy as np
+
+
+class CustomEncoder(json.JSONEncoder):
+    def default(self, obj):
+        """If input object is an ndarray it will be converted into a dict
+        holding dtype, shape and the data, base64 encoded.
+        """
+        numpy_types = (
+            np.bool_,
+            # np.bytes_, -- python `bytes` class is not json serializable
+            # np.complex64,  -- python `complex` class is not json serializable
+            # np.complex128,  -- python `complex` class is not json serializable
+            # np.complex256,  -- python `complex` class is not json serializable
+            # np.datetime64,  -- python `datetime.datetime` class is not json serializable

Review comment:
       datetimes are quite common in our data - I wonder if we can convert to string?

##########
File path: analysis/webservice/algorithms/NexusCalcHandler.py
##########
@@ -0,0 +1,70 @@
+import time
+import types
+
+from nexustiles.nexustiles import NexusTileService
+
+
+class NexusCalcHandler(object):
+    @classmethod
+    def validate(cls):
+        if "calc" not in cls.__dict__ or not type(cls.__dict__["calc"]) == types.FunctionType:
+            raise Exception("Method 'calc' has not been declared")
+
+        if "path" not in cls.__dict__:
+            raise Exception("Property 'path' has not been defined")
+
+        if "name" not in cls.__dict__:
+            raise Exception("Property 'name' has not been defined")
+
+        if "description" not in cls.__dict__:
+            raise Exception("Property 'description' has not been defined")
+
+        if "params" not in cls.__dict__:
+            raise Exception("Property 'params' has not been defined")
+
+    def __init__(self, skipCassandra=False, skipSolr=False):
+
+        self._tile_service = NexusTileService(skipCassandra, skipSolr)
+
+    def calc(self, computeOptions, **args):
+        raise Exception("calc() not yet implemented")
+
+    def _mergeDicts(self, x, y):
+        z = x.copy()
+        z.update(y)
+        return z
+
+    def _now(self):
+        millis = int(round(time.time() * 1000))
+        return millis
+
+    def _mergeDataSeries(self, resultsData, dataNum, resultsMap):
+
+        for entry in resultsData:
+
+            # frmtdTime = datetime.fromtimestamp(entry["time"] ).strftime("%Y-%m")
+            frmtdTime = entry["time"]
+
+            if not frmtdTime in resultsMap:
+                resultsMap[frmtdTime] = []
+            entry["ds"] = dataNum
+            resultsMap[frmtdTime].append(entry)
+
+    def _resultsMapToList(self, resultsMap):
+        resultsList = []
+        for key, value in resultsMap.iteritems():
+            resultsList.append(value)
+
+        resultsList = sorted(resultsList, key=lambda entry: entry[0]["time"])
+        return resultsList

Review comment:
       I'm curious why we're using a 2-tuple here (maybe Joe can elaborate)

##########
File path: analysis/webservice/algorithms_spark/CorrMapSpark.py
##########
@@ -22,22 +22,17 @@
 from nexustiles.nexustiles import NexusTileService
 
 # from time import time
-from webservice.NexusHandler import nexus_handler, SparkHandler, DEFAULT_PARAMETERS_SPEC
+from webservice.NexusHandler import nexus_handler, DEFAULT_PARAMETERS_SPEC
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo, lines 26, 31 

##########
File path: analysis/webservice/algorithms_spark/DailyDifferenceAverageSpark.py
##########
@@ -23,7 +23,8 @@
 from shapely import wkt
 from shapely.geometry import Polygon
 
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo lines 27, 41, 75

##########
File path: analysis/webservice/algorithms_spark/ClimMapSpark.py
##########
@@ -22,22 +22,17 @@
 import numpy as np
 from nexustiles.nexustiles import NexusTileService
 
-from webservice.NexusHandler import nexus_handler, SparkHandler, DEFAULT_PARAMETERS_SPEC
+from webservice.NexusHandler import nexus_handler, DEFAULT_PARAMETERS_SPEC
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 
 @nexus_handler
-class ClimMapSparkHandlerImpl(SparkHandler):
+class ClimMapNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       typo

##########
File path: analysis/webservice/algorithms_spark/ClimMapSpark.py
##########
@@ -66,13 +61,13 @@ def _map(tile_in_spark):
             # print 'nexus call start at time %f' % t1
             # sys.stdout.flush()
             nexus_tiles = \
-                ClimMapSparkHandlerImpl.query_by_parts(tile_service,
-                                                       min_lat, max_lat,
-                                                       min_lon, max_lon,
-                                                       ds,
-                                                       t_start,
-                                                       t_end,
-                                                       part_dim=2)
+                ClimMapNexusSkarkHandlerImpl.query_by_parts(tile_service,

Review comment:
       typo

##########
File path: analysis/webservice/algorithms_spark/TimeSeriesSpark.py
##########
@@ -32,7 +32,8 @@
 from pytz import timezone
 from scipy import stats
 from webservice import Filtering as filtering
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler

Review comment:
       typo lines 36, 46, 249

##########
File path: analysis/webservice/webapp.py
##########
@@ -203,22 +73,21 @@ def async_callback(self, result):
 
     spark_context = None
     for clazzWrapper in NexusHandler.AVAILABLE_HANDLERS:
-        if issubclass(clazzWrapper.clazz(), NexusHandler.SparkHandler):
+        if issubclass(clazzWrapper, webservice.algorithms_spark.NexusCalcSparkHandler.NexusCalcSkarkHandler):

Review comment:
       typo Skark




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r451855059



##########
File path: analysis/webservice/algorithms/NexusCalcHandler.py
##########
@@ -0,0 +1,70 @@
+import time
+import types
+
+from nexustiles.nexustiles import NexusTileService
+
+
+class NexusCalcHandler(object):
+    @classmethod
+    def validate(cls):
+        if "calc" not in cls.__dict__ or not type(cls.__dict__["calc"]) == types.FunctionType:
+            raise Exception("Method 'calc' has not been declared")
+
+        if "path" not in cls.__dict__:
+            raise Exception("Property 'path' has not been defined")
+
+        if "name" not in cls.__dict__:
+            raise Exception("Property 'name' has not been defined")
+
+        if "description" not in cls.__dict__:
+            raise Exception("Property 'description' has not been defined")
+
+        if "params" not in cls.__dict__:
+            raise Exception("Property 'params' has not been defined")
+
+    def __init__(self, skipCassandra=False, skipSolr=False):
+
+        self._tile_service = NexusTileService(skipCassandra, skipSolr)
+
+    def calc(self, computeOptions, **args):
+        raise Exception("calc() not yet implemented")
+
+    def _mergeDicts(self, x, y):
+        z = x.copy()
+        z.update(y)
+        return z
+
+    def _now(self):
+        millis = int(round(time.time() * 1000))
+        return millis
+
+    def _mergeDataSeries(self, resultsData, dataNum, resultsMap):
+
+        for entry in resultsData:
+
+            # frmtdTime = datetime.fromtimestamp(entry["time"] ).strftime("%Y-%m")
+            frmtdTime = entry["time"]
+
+            if not frmtdTime in resultsMap:
+                resultsMap[frmtdTime] = []
+            entry["ds"] = dataNum
+            resultsMap[frmtdTime].append(entry)
+
+    def _resultsMapToList(self, resultsMap):
+        resultsList = []
+        for key, value in resultsMap.iteritems():
+            resultsList.append(value)
+
+        resultsList = sorted(resultsList, key=lambda entry: entry[0]["time"])
+        return resultsList

Review comment:
       Good question. I don't know. I would not change that as part of this pull request.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] eamonford commented on a change in pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
eamonford commented on a change in pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#discussion_r446363886



##########
File path: analysis/webservice/algorithms_spark/ClimMapSpark.py
##########
@@ -22,22 +22,17 @@
 import numpy as np
 from nexustiles.nexustiles import NexusTileService
 
-from webservice.NexusHandler import nexus_handler, SparkHandler, DEFAULT_PARAMETERS_SPEC
+from webservice.NexusHandler import nexus_handler, DEFAULT_PARAMETERS_SPEC
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 
 @nexus_handler
-class ClimMapSparkHandlerImpl(SparkHandler):
+class ClimMapNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       typo, lol

##########
File path: analysis/webservice/algorithms_spark/HofMoellerSpark.py
##########
@@ -100,7 +101,7 @@ def hofmoeller_stats(metrics_callback, tile_in_spark):
         return stats
 
 
-class BaseHoffMoellerHandlerImpl(SparkHandler):
+class BaseHoffMoellerSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       Skark

##########
File path: analysis/webservice/algorithms_spark/DailyDifferenceAverageSpark.py
##########
@@ -37,7 +38,7 @@ def iso_time_to_epoch(str_time):
 
 
 @nexus_handler
-class DailyDifferenceAverageSparkImpl(SparkHandler):
+class DailyDifferenceAverageNexusImplSkark(NexusCalcSkarkHandler):

Review comment:
       typo: Skark

##########
File path: analysis/webservice/algorithms_spark/NexusCalcSparkHandler.py
##########
@@ -0,0 +1,359 @@
+import logging
+
+import numpy as np
+from netCDF4._netCDF4 import Dataset
+from webservice.algorithms.NexusCalcHandler import NexusCalcHandler
+from webservice.metrics import MetricsRecord, SparkAccumulatorMetricsField, NumberMetricsField
+from webservice.webmodel import NexusProcessingException
+
+
+class NexusCalcSkarkHandler(NexusCalcHandler):

Review comment:
       Skark

##########
File path: analysis/webservice/algorithms_spark/TimeAvgMapSpark.py
##########
@@ -21,15 +21,18 @@
 import shapely.geometry
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 EPOCH = timezone('UTC').localize(datetime(1970, 1, 1))
 ISO_8601 = '%Y-%m-%dT%H:%M:%S%z'
 
 
 @nexus_handler
-class TimeAvgMapSparkHandlerImpl(SparkHandler):
+class TimeAvgMapNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       Skark

##########
File path: analysis/webservice/webmodel/NexusRequestObjectTornadoFree.py
##########
@@ -0,0 +1,108 @@
+import logging
+import re
+from datetime import datetime
+from decimal import Decimal
+
+from pytz import UTC
+from webservice.webmodel.RequestParameters import RequestParameters
+from webservice.webmodel.StatsComputeOptions import StatsComputeOptions
+
+
+class NexusRequestObjectTornadoFree(StatsComputeOptions):

Review comment:
       Will this be renamed once NexusRequestObject is phased out?

##########
File path: analysis/webservice/algorithms_spark/TimeSeriesSpark.py
##########
@@ -42,7 +43,7 @@
 
 
 @nexus_handler
-class TimeSeriesHandlerImpl(SparkHandler):
+class TimeSeriesSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       Skark

##########
File path: analysis/webservice/algorithms_spark/VarianceSpark.py
##########
@@ -23,15 +23,16 @@
 from nexustiles.nexustiles import NexusTileService
 from pytz import timezone
 
-from webservice.NexusHandler import nexus_handler, SparkHandler
+from webservice.NexusHandler import nexus_handler
+from webservice.algorithms_spark.NexusCalcSparkHandler import NexusCalcSkarkHandler
 from webservice.webmodel import NexusResults, NexusProcessingException, NoDataException
 
 EPOCH = timezone('UTC').localize(datetime(1970, 1, 1))
 ISO_8601 = '%Y-%m-%dT%H:%M:%S%z'
 
 
 @nexus_handler
-class VarianceSparkHandlerImpl(SparkHandler):
+class VarianceNexusSkarkHandlerImpl(NexusCalcSkarkHandler):

Review comment:
       Skark




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] echyam commented on pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
echyam commented on pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#issuecomment-650433143


   Sorry my review took so long.  Looks like all the typos got fixed, but I think we should still generalize from SLCP env URLs if possible.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-sdap-nexus] tloubrieu-jpl commented on pull request #104: Sdap 252

Posted by GitBox <gi...@apache.org>.
tloubrieu-jpl commented on pull request #104:
URL: https://github.com/apache/incubator-sdap-nexus/pull/104#issuecomment-655800182


   I also compared performances on podaac dev of the new docker image for this pull request vs nexus-webapp:distributed.0.1.3 
   
   They are similar. A bit better with the pull request development, for what it is worth (none of them are good):
   
     | new image | new image | previous image
   -- | -- | -- | --
   area_average_time_series_time | 60.0904815 | 32.035773 | 71.1388659
   latitude_hovmoller_time | 15.8122313 | 32.9109372 | 39.7897524
   longitude_hovmoller_time | 9.77846225 | 14.6263703 | 16.9613255
   time_average_map_time | 39.7967503 | 66.5912247 | 132.412795
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org