You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sis.apache.org by de...@apache.org on 2022/01/28 19:43:22 UTC

[sis] branch geoapi-4.0 updated (d8b9343 -> 33d5d46)

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

desruisseaux pushed a change to branch geoapi-4.0
in repository https://gitbox.apache.org/repos/asf/sis.git.


    from d8b9343  Move some metadata fields in a separated `ImageMetadataBuilder` class. The goal is to make their life cycle more visible, especially `XMLMetadata` which causes confusing metadata tree if not merged last.
     new afc247a  Add a `Citations.WMO` constant for the World Meteorological Organization.
     new 9dde363  Provide a way to alter parameter values of inverse operation in a call to `ContextualParameters.inverse(…)`.
     new 33d5d46  Add "Rotated Latitude/Longitude" coordinate operation.

The 3 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.


Summary of changes:
 .../sis/metadata/iso/citation/Citations.java       |  14 +-
 .../sis/metadata/iso/citation/package-info.java    |   2 +-
 .../apache/sis/metadata/sql/MetadataFallback.java  |   9 +-
 .../org/apache/sis/metadata/sql/Citations.sql      |   9 +-
 .../referencing/provider/RotatedNorthPole.java     | 150 +++++++++
 .../referencing/provider/RotatedSouthPole.java     | 154 +++++++++
 .../referencing/provider/package-info.java         |   2 +-
 .../operation/transform/ContextualParameters.java  | 107 ++++--
 .../transform/CoordinateSystemTransform.java       |   2 +-
 .../transform/EllipsoidToCentricTransform.java     |   6 +-
 .../operation/transform/RotatedPole.java           | 357 +++++++++++++++++++++
 .../operation/transform/package-info.java          |   2 +-
 ...g.opengis.referencing.operation.OperationMethod |   2 +
 .../referencing/provider/ProvidersTest.java        |   4 +-
 .../operation/transform/RotatedPoleTest.java       | 118 +++++++
 .../sis/test/suite/ReferencingTestSuite.java       |   1 +
 16 files changed, 897 insertions(+), 42 deletions(-)
 create mode 100644 core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedNorthPole.java
 create mode 100644 core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedSouthPole.java
 create mode 100644 core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/RotatedPole.java
 create mode 100644 core/sis-referencing/src/test/java/org/apache/sis/referencing/operation/transform/RotatedPoleTest.java

[sis] 02/03: Provide a way to alter parameter values of inverse operation in a call to `ContextualParameters.inverse(…)`.

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

desruisseaux pushed a commit to branch geoapi-4.0
in repository https://gitbox.apache.org/repos/asf/sis.git

commit 9dde363825909c92cf7ac887931c498e9586bee3
Author: Martin Desruisseaux <ma...@geomatys.com>
AuthorDate: Mon Jan 24 12:05:00 2022 +0100

    Provide a way to alter parameter values of inverse operation
    in a call to `ContextualParameters.inverse(…)`.
---
 .../operation/transform/ContextualParameters.java  | 107 +++++++++++++++------
 .../transform/CoordinateSystemTransform.java       |   2 +-
 .../transform/EllipsoidToCentricTransform.java     |   6 +-
 .../operation/transform/package-info.java          |   2 +-
 4 files changed, 82 insertions(+), 35 deletions(-)

diff --git a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/ContextualParameters.java b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/ContextualParameters.java
index 853cfaa..4e1e01d 100644
--- a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/ContextualParameters.java
+++ b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/ContextualParameters.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.Objects;
+import java.util.function.BiPredicate;
 import java.io.Serializable;
 import org.opengis.util.FactoryException;
 import org.opengis.parameter.GeneralParameterValue;
@@ -54,6 +55,7 @@ import org.apache.sis.io.wkt.Formatter;
 import org.apache.sis.util.logging.Logging;
 import org.apache.sis.util.resources.Errors;
 import org.apache.sis.util.ArgumentChecks;
+import org.apache.sis.util.ArraysExt;
 
 
 /**
@@ -121,7 +123,7 @@ import org.apache.sis.util.ArgumentChecks;
  * Serialization should be used only for short term storage or RMI between applications running the same SIS version.
  *
  * @author  Martin Desruisseaux (Geomatys)
- * @version 1.0
+ * @version 1.2
  *
  * @see org.apache.sis.referencing.operation.projection.NormalizedProjection
  * @see AbstractMathTransform#getContextualParameters()
@@ -205,12 +207,17 @@ public class ContextualParameters extends Parameters implements Serializable {
      *
      * <p>This array is modifiable after construction, but is considered unmodifiable after
      * {@link #completeTransform(MathTransformFactory, MathTransform)} has been invoked.</p>
+     *
+     * @see #parameter(String)
+     * @see #freeze()
      */
     private ParameterValue<?>[] values;
 
     /**
      * If the inverse coordinate operation can be described by another {@code ContextualParameters} instance,
      * a reference to that instance. Otherwise {@code null}.
+     *
+     * @see #inverse(ParameterDescriptorGroup, BiPredicate)
      */
     private ContextualParameters inverse;
 
@@ -240,7 +247,10 @@ public class ContextualParameters extends Parameters implements Serializable {
      * See class javadoc for more information.
      *
      * @param  method  the non-linear operation method for which to define the parameter values.
+     *
+     * @deprecated Use the constructor with explicit number of dimensions instead.
      */
+    @Deprecated
     public ContextualParameters(final OperationMethod method) {
         ArgumentChecks.ensureNonNull("method", method);
         descriptor  = method.getParameters();
@@ -270,18 +280,42 @@ public class ContextualParameters extends Parameters implements Serializable {
     }
 
     /**
-     * Creates a {@code ContextualParameters} for the inverse operation.
+     * Creates a new and frozen {@code ContextualParameters} for the inverse operation.
+     * An optional {@code mapper} can be specified for setting the parameter values.
+     * If the mapper is-non null, then it will be invoked for each new parameters.
+     * The first argument given to the mapper is the {@code forward} operation,
+     * and the second argument is the parameter to configure.
+     * The return value is whether the parameter should be kept in the new operation.
      *
      * @param  desc     descriptor of the inverse operation.
      * @param  forward  the parameters created for the forward operation.
+     * @param  mapper   the function to invoke for setting parameter values,
+     *                  or {@code null} if the inverse operation uses the same parameter values.
+     *
+     * @see #inverse(ParameterDescriptorGroup, BiPredicate)
      */
-    private ContextualParameters(final ParameterDescriptorGroup desc, final ContextualParameters forward) {
+    private ContextualParameters(final ParameterDescriptorGroup desc, final ContextualParameters forward,
+                                 final BiPredicate<Parameters, ParameterValue<?>> mapper)
+    {
         descriptor  = desc;
+        inverse     = forward;
         normalize   = forward.getMatrix(MatrixRole.INVERSE_DENORMALIZATION);
         denormalize = forward.getMatrix(MatrixRole.INVERSE_NORMALIZATION);
-        values      = forward.values;
-        inverse     = forward;
-        isFrozen    = true;
+        if (mapper == null) {
+            values = forward.values;
+        } else {
+            final List<GeneralParameterDescriptor> descriptors = desc.descriptors();
+            final ParameterValue<?>[] values = new ParameterValue<?>[descriptors.size()];
+            int count = 0;
+            for (int i=0; i < values.length; i++) {
+                final ContextualParameter<?> p = new ContextualParameter<>((ParameterDescriptor<?>) descriptors.get(i));
+                if (mapper.test(forward, p)) {
+                    values[count++] = p;
+                }
+            }
+            this.values = ArraysExt.resize(values, count);
+        }
+        isFrozen = true;
     }
 
     /**
@@ -298,17 +332,29 @@ public class ContextualParameters extends Parameters implements Serializable {
     }
 
     /**
-     * Creates a {@code ContextualParameters} for the inverse operation.
-     *
-     * @param  desc  descriptor of the inverse operation.
+     * Creates a new and frozen {@code ContextualParameters} for the inverse of this operation.
+     * An optional {@code mapper} can be specified for setting the parameter values.
+     * If the mapper is-non null, then it will be invoked for each new parameters.
+     * The first argument given to the mapper will be {@code this} operation,
+     * and the second argument is the parameter to configure.
+     * The return value is whether the parameter should be kept in the new operation.
+     *
+     * <p>This method caches the inverse operation. It is caller responsibility to ensure that all
+     * arguments given to this method are constants for a given {@link MathTransform} instance.</p>
+     *
+     * @param  desc    descriptor of the inverse operation.
+     * @param  mapper  the function to invoke for setting parameter values,
+     *                 or {@code null} if the inverse operation uses the same parameter values.
      * @return parameters for the inverse operation.
      */
-    final synchronized ContextualParameters inverse(final ParameterDescriptorGroup desc) {
+    final synchronized ContextualParameters inverse(final ParameterDescriptorGroup desc,
+            final BiPredicate<Parameters, ParameterValue<?>> mapper)
+    {
         if (inverse == null) {
             if (!isFrozen) {
                 freeze();
             }
-            inverse = new ContextualParameters(desc, this);
+            inverse = new ContextualParameters(desc, this, mapper);
         }
         assert inverse.descriptor == desc;
         return inverse;
@@ -412,7 +458,7 @@ public class ContextualParameters extends Parameters implements Serializable {
         }
         /*
          * Following must be outside the synchronized block in order to avoid potential deadlock while invoking
-         * inverse.getMatrix(role). We do not cache the matrix here, but 'inverse' is likely to have cached it.
+         * inverse.getMatrix(role). We do not cache the matrix here, but `inverse` is likely to have cached it.
          */
         final Matrix m;
         if (inverse != null) {
@@ -527,8 +573,8 @@ public class ContextualParameters extends Parameters implements Serializable {
         }
         /*
          * Following call must be outside the synchronized block for avoiding dead-lock. This is because the
-         * factory typically contains a WeakHashSet, which invoke in turn the 'equals' methods in this class
-         * (indirectly, through 'kernel' comparison). We need to be outside the synchronized block for having
+         * factory typically contains a WeakHashSet, which invoke in turn the `equals` methods in this class
+         * (indirectly, through `kernel` comparison). We need to be outside the synchronized block for having
          * the locks taken in same order (WeakHashSet lock before the ContextualParameters lock).
          */
         return factory.createConcatenatedTransform(factory.createConcatenatedTransform(n, kernel), d);
@@ -543,7 +589,7 @@ public class ContextualParameters extends Parameters implements Serializable {
         isFrozen = true;
         /*
          * Sort the parameter values in the same order than the parameter descriptor. This is not essential,
-         * but makes easier to read 'toString()' output by ensuring a consistent order for most projections.
+         * but makes easier to read `toString()` output by ensuring a consistent order for most projections.
          * Some WKT parsers other than SIS may also require the parameter values to be listed in that specific
          * order. We proceed by first copying all parameters in a temporary HashMap:
          */
@@ -560,11 +606,11 @@ public class ContextualParameters extends Parameters implements Serializable {
             }
         }
         /*
-         * Then, copy all HashMap values back to the 'values' array in the order they are declared in the
+         * Then, copy all HashMap values back to the `values` array in the order they are declared in the
          * descriptor. Implementation note: the iteration termination condition uses the values array, not
          * the descriptors list, because the former is often shorter than the later. We should never reach
-         * the end of descriptors list before the end of values array because 'descriptors' contains all
-         * 'parameters' keys. This is verified by the 'assert' below.
+         * the end of descriptors list before the end of values array because `descriptors` contains all
+         * `parameters` keys. This is verified by the `assert` below.
          */
         values = new ParameterValue<?>[parameters.size()];
         assert descriptor.descriptors().containsAll(parameters.keySet());
@@ -572,8 +618,8 @@ public class ContextualParameters extends Parameters implements Serializable {
         for (int i=0; i < values.length;) {
             /*
              * No need to check for it.hasNext(), since a NoSuchElementException below would be a bug in
-             * our algorithm (or a concurrent change in the 'descriptor.descriptors()' list, which would
-             * be a contract violation). See above 'assert'.
+             * our algorithm (or a concurrent change in the `descriptor.descriptors()` list, which would
+             * be a contract violation). See above `assert`.
              */
             final ParameterValue<?> p = parameters.get(it.next());
             if (p != null) {
@@ -610,6 +656,7 @@ public class ContextualParameters extends Parameters implements Serializable {
         for (int i=0; i < values.length; i++) {
             ParameterValue<?> p = values[i];
             if (p == null) {
+                // Null values are always last, so it is okay to stop the search here.
                 values[i] = p = new ContextualParameter<>((ParameterDescriptor<?>) desc);
                 return p;
             }
@@ -618,7 +665,7 @@ public class ContextualParameters extends Parameters implements Serializable {
             }
         }
         /*
-         * We may reach this point if map projection construction is completed (i.e. 'completeTransform(…)' has
+         * We may reach this point if map projection construction is completed (i.e. `completeTransform(…)` has
          * been invoked) and the user asks for a parameter which is not one of the parameters that we retained.
          * Returns a parameter initialized to the default value, which is the actual value (otherwise we would
          * have stored that parameter).  Note: we do not bother making the parameter immutable for performance
@@ -868,14 +915,14 @@ public class ContextualParameters extends Parameters implements Serializable {
          * is often for changing axis order. Thanks to double-double arithmetic in SIS matrices,
          * the non-zero values are usually accurate. But the values that should be zero are much
          * harder to get right. Sometime we see small values (around 1E-12) in the last column of
-         * the 'before' matrix below. Since this column contains translation terms, those numbers
+         * the `before` matrix below. Since this column contains translation terms, those numbers
          * are in the unit of measurement of input values of the MathTransform after the matrix.
          *
          *   - For forward map projections, those values are conceptually in decimal degrees
-         *     (in fact the values are converted to radians but not by this 'before' matrix).
+         *     (in fact the values are converted to radians but not by this `before` matrix).
          *
          *   - For inverse map projections, those values are conceptually in metres (in fact
-         *     converted to distances on a unitary ellipsoid but not by this 'before' matrix).
+         *     converted to distances on a unitary ellipsoid but not by this `before` matrix).
          *
          *   - Geographic/Geocentric transformations behave like map projections in regard to units.
          *     Molodensky transformations conceptually use always decimal degrees. There is not much
@@ -883,7 +930,7 @@ public class ContextualParameters extends Parameters implements Serializable {
          *
          * Consequently we set the tolerance threshold to ANGULAR_TOLERANCE. We do not bother (at least
          * for now) to identify the cases where we could use LINEAR_TOLERANCE because just checking the
-         * 'inverse' flag is not sufficient (e.g. the Molodensky case). Since the angular tolerance is
+         * `inverse` flag is not sufficient (e.g. the Molodensky case). Since the angular tolerance is
          * smaller than the linear one, unconditional usage of ANGULAR_TOLERANCE is more conservative.
          */
         before = Matrices.isIdentity(userDefined, Formulas.ANGULAR_TOLERANCE) ? null : userDefined;
@@ -902,20 +949,20 @@ public class ContextualParameters extends Parameters implements Serializable {
             userDefined = Matrices.multiply(after, userDefined);
         }
         /*
-         * Note on rounding error: same discussion than the "note on rounding error" of the 'before' matrix,
+         * Note on rounding error: same discussion than the "note on rounding error" of the `before` matrix,
          * with the following differences:
          *
          *   - For forward map projections, unit of measurements of translation terms are conceptually
-         *     metres (instead of degrees) multiplied by the scale factors in the 'after' matrix.
+         *     metres (instead of degrees) multiplied by the scale factors in the `after` matrix.
          *
          *   - For inverse map projections, unit of measurements of translation terms are conceptually
-         *     degrees (instead of metres) multiplied by the scale factors in the 'after' matrix.
+         *     degrees (instead of metres) multiplied by the scale factors in the `after` matrix.
          *
          *   - And so on for all cases: swap the units of the forward and inverse cases, then multiply
-         *     by the scale factor. Note that the multiplication step does not exist in the 'before' case.
+         *     by the scale factor. Note that the multiplication step does not exist in the `before` case.
          *
          * Since we are seeking for the identity matrix, the scale factor is 1. We do not bother to distinguish
-         * the ANGULAR_TOLERANCE and LINEAR_TOLERANCE cases for the same reasons than for the 'before' matrix.
+         * the ANGULAR_TOLERANCE and LINEAR_TOLERANCE cases for the same reasons than for the `before` matrix.
          */
         after = Matrices.isIdentity(userDefined, Formulas.ANGULAR_TOLERANCE) ? null : userDefined;
         /*
diff --git a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/CoordinateSystemTransform.java b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/CoordinateSystemTransform.java
index ae211e3..6a6390e 100644
--- a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/CoordinateSystemTransform.java
+++ b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/CoordinateSystemTransform.java
@@ -44,7 +44,7 @@ import org.apache.sis.referencing.operation.DefaultOperationMethod;
 
 /**
  * Base class of conversions between coordinate systems.
- * Each subclasses should have a singleton instance.
+ * Each subclass should have a singleton instance.
  *
  * @author  Martin Desruisseaux (Geomatys)
  * @version 1.1
diff --git a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/EllipsoidToCentricTransform.java b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/EllipsoidToCentricTransform.java
index d4d4ed2..0b5dbd1 100644
--- a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/EllipsoidToCentricTransform.java
+++ b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/EllipsoidToCentricTransform.java
@@ -815,7 +815,7 @@ next:   while (--numPts >= 0) {
          */
         @Override
         protected ContextualParameters getContextualParameters() {
-            return forward.context.inverse(GeocentricToGeographic.PARAMETERS);
+            return forward.context.inverse(GeocentricToGeographic.PARAMETERS, null);
         }
 
         /**
@@ -842,8 +842,8 @@ next:   while (--numPts >= 0) {
         @Debug
         @Override
         public ParameterDescriptorGroup getParameterDescriptors() {
-            return new DefaultParameterDescriptorGroup(Collections.singletonMap(ParameterDescriptorGroup.NAME_KEY,
-                            new ImmutableIdentifier(Citations.SIS, Constants.SIS, "Centric to ellipsoid (radians domain)")),
+            ImmutableIdentifier name = new ImmutableIdentifier(Citations.SIS, Constants.SIS, "Centric to ellipsoid (radians domain)");
+            return new DefaultParameterDescriptorGroup(Collections.singletonMap(ParameterDescriptorGroup.NAME_KEY, name),
                     forward.getParameterDescriptors());
         }
 
diff --git a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/package-info.java b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/package-info.java
index db70c02..5298191 100644
--- a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/package-info.java
+++ b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/package-info.java
@@ -61,7 +61,7 @@
  *
  * @author  Martin Desruisseaux (IRD, Geomatys)
  * @author  Adrian Custer (Geomatys)
- * @version 1.1
+ * @version 1.2
  * @since   0.5
  * @module
  */

[sis] 01/03: Add a `Citations.WMO` constant for the World Meteorological Organization.

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

desruisseaux pushed a commit to branch geoapi-4.0
in repository https://gitbox.apache.org/repos/asf/sis.git

commit afc247ae5d9d418ca9626789c1e5ef341894b06d
Author: Martin Desruisseaux <ma...@geomatys.com>
AuthorDate: Mon Jan 24 12:02:12 2022 +0100

    Add a `Citations.WMO` constant for the World Meteorological Organization.
---
 .../org/apache/sis/metadata/iso/citation/Citations.java    | 14 +++++++++++++-
 .../org/apache/sis/metadata/iso/citation/package-info.java |  2 +-
 .../java/org/apache/sis/metadata/sql/MetadataFallback.java |  9 ++++++++-
 .../resources/org/apache/sis/metadata/sql/Citations.sql    |  9 +++++++--
 4 files changed, 29 insertions(+), 5 deletions(-)

diff --git a/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/Citations.java b/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/Citations.java
index 09b203b..c3a74c6 100644
--- a/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/Citations.java
+++ b/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/Citations.java
@@ -75,7 +75,7 @@ import org.apache.sis.metadata.iso.DefaultIdentifier;           // For javadoc
  * </ul>
  *
  * @author  Martin Desruisseaux (IRD, Geomatys)
- * @version 1.1
+ * @version 1.2
  * @since   0.3
  * @module
  */
@@ -294,6 +294,17 @@ public final class Citations extends Static {
     public static final IdentifierSpace<String> ESRI = new CitationConstant.Authority<>("ArcGIS", "ESRI");
 
     /**
+     * The authority for identifiers of objects defined by the
+     * <a href="https://www.wmo.int">World Meteorological Organization</a>.
+     * The {@linkplain IdentifierSpace#getName() name} of this identifier space is fixed to {@code "WMO"}.
+     * This citation is used as the authority for some coordinate operations other than EPSG and ESRI ones,
+     * for example "Rotated latitude/longitude".
+     *
+     * @since 1.2
+     */
+    public static final IdentifierSpace<String> WMO = new CitationConstant.Authority<>("WMO");
+
+    /**
      * The authority for identifiers of objects defined by the netCDF specification.
      * The {@linkplain IdentifierSpace#getName() name} of this identifier space is fixed to {@code "NetCDF"}.
      * This citation is used as the authority for some map projection method and parameter names
@@ -431,6 +442,7 @@ public final class Citations extends Static {
         (CitationConstant) OGC,
         (CitationConstant) WMS,                 // Must be after OGC because it declares the same namespace.
         (CitationConstant) ESRI,
+        (CitationConstant) WMO,
         (CitationConstant) NETCDF,
         (CitationConstant) GEOTIFF,
         (CitationConstant) PROJ4,
diff --git a/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/package-info.java b/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/package-info.java
index 28e6bca..9d4e0fb 100644
--- a/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/package-info.java
+++ b/core/sis-metadata/src/main/java/org/apache/sis/metadata/iso/citation/package-info.java
@@ -87,7 +87,7 @@
  * @author  Touraïvane (IRD)
  * @author  Cédric Briançon (Geomatys)
  * @author  Cullen Rombach (Image Matters)
- * @version 1.0
+ * @version 1.2
  * @since   0.3
  * @module
  */
diff --git a/core/sis-metadata/src/main/java/org/apache/sis/metadata/sql/MetadataFallback.java b/core/sis-metadata/src/main/java/org/apache/sis/metadata/sql/MetadataFallback.java
index ae5a986..b094e70 100644
--- a/core/sis-metadata/src/main/java/org/apache/sis/metadata/sql/MetadataFallback.java
+++ b/core/sis-metadata/src/main/java/org/apache/sis/metadata/sql/MetadataFallback.java
@@ -37,7 +37,7 @@ import static java.util.Collections.singleton;
  * Used when connection to the spatial metadata can not be established.
  *
  * @author  Martin Desruisseaux (Geomatys)
- * @version 1.0
+ * @version 1.2
  * @since   1.0
  * @module
  */
@@ -149,6 +149,13 @@ final class MetadataFallback extends MetadataSource {
                 presentationForm      = PresentationForm.DOCUMENT_DIGITAL;
                 break;
             }
+            case "WMO": {
+                title                 = "WMO Information System (WIS)";
+                code                  = key;
+                citedResponsibleParty = "World Meteorological Organization";
+                presentationForm      = PresentationForm.DOCUMENT_DIGITAL;
+                break;
+            }
             case "IOGP": {          // Not in public API (see Citations.IOGP javadoc)
                 title = "IOGP Surveying and Positioning Guidance Note 7";
                 code             = Constants.IOGP;
diff --git a/core/sis-metadata/src/main/resources/org/apache/sis/metadata/sql/Citations.sql b/core/sis-metadata/src/main/resources/org/apache/sis/metadata/sql/Citations.sql
index 3c010e3..392881c 100644
--- a/core/sis-metadata/src/main/resources/org/apache/sis/metadata/sql/Citations.sql
+++ b/core/sis-metadata/src/main/resources/org/apache/sis/metadata/sql/Citations.sql
@@ -50,6 +50,7 @@ INSERT INTO metadata."OnlineResource" ("ID", "linkage") VALUES
   ('PostGIS', 'https://postgis.net/'),
   ('Proj4',   'https://proj4.org/'),
   ('SIS',     'https://sis.apache.org/'),
+  ('WMO',     'https://www.wmo.int/'),
   ('WMS',     'https://www.ogc.org/standards/wms');
 
 UPDATE metadata."OnlineResource" SET "function" = 'information';
@@ -105,7 +106,8 @@ INSERT INTO metadata."Organisation" ("ID", "name") VALUES
   ('{org}NATO',   'North Atlantic Treaty Organization'),
   ('{org}OGC',    'Open Geospatial Consortium'),
   ('{org}OSGeo',  'The Open Source Geospatial Foundation'),
-  ('{org}PBI',    'Pitney Bowes Inc.');
+  ('{org}PBI',    'Pitney Bowes Inc.'),
+  ('{org}WMO',    'World Meteorological Organization');
 
 INSERT INTO metadata."Responsibility" ("ID", "party", "role") VALUES
   ('Apache',  '{org}Apache', 'resourceProvider'),
@@ -118,7 +120,8 @@ INSERT INTO metadata."Responsibility" ("ID", "party", "role") VALUES
   ('MapInfo', '{org}PBI',    'principalInvestigator'),
   ('NATO',    '{org}NATO',   'principalInvestigator'),
   ('OGC',     '{org}OGC',    'principalInvestigator'),
-  ('OSGeo',   '{org}OSGeo',  'resourceProvider');
+  ('OSGeo',   '{org}OSGeo',  'resourceProvider'),
+  ('WMO',     '{org}WMO',    'principalInvestigator');
 
 
 
@@ -219,10 +222,12 @@ INSERT INTO metadata."Citation" ("ID", "onlineResource", "edition", "citedRespon
 --
 INSERT INTO metadata."Identifier" ("ID", "code") VALUES
   ('OGC',  'OGC'),
+  ('WMO',  'WMO'),
   ('IOGP', 'IOGP');
 
 INSERT INTO metadata."Citation" ("ID", "onlineResource", "citedResponsibleParty", "presentationForm", "title") VALUES
   ('OGC',  'OGCNA', 'OGC',  'documentDigital', 'OGC Naming Authority'),
+  ('WMO',  'WMO',   'WMO',  'documentDigital', 'WMO Information System (WIS)'),
   ('IOGP', 'IOGP',  'IOGP', 'documentDigital', 'IOGP Surveying and Positioning Guidance Note 7');
 
 UPDATE metadata."Citation" SET "identifier" = "ID" WHERE "ID"<>'ISBN' AND "ID"<>'ISSN' AND "ID"<>'MGRS';

[sis] 03/03: Add "Rotated Latitude/Longitude" coordinate operation.

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

desruisseaux pushed a commit to branch geoapi-4.0
in repository https://gitbox.apache.org/repos/asf/sis.git

commit 33d5d46b7cbcdb392c6a2c140b204f708bbb6389
Author: Martin Desruisseaux <ma...@geomatys.com>
AuthorDate: Mon Jan 24 12:06:56 2022 +0100

    Add "Rotated Latitude/Longitude" coordinate operation.
---
 .../referencing/provider/RotatedNorthPole.java     | 150 +++++++++
 .../referencing/provider/RotatedSouthPole.java     | 154 +++++++++
 .../referencing/provider/package-info.java         |   2 +-
 .../operation/transform/RotatedPole.java           | 357 +++++++++++++++++++++
 ...g.opengis.referencing.operation.OperationMethod |   2 +
 .../referencing/provider/ProvidersTest.java        |   4 +-
 .../operation/transform/RotatedPoleTest.java       | 118 +++++++
 .../sis/test/suite/ReferencingTestSuite.java       |   1 +
 8 files changed, 786 insertions(+), 2 deletions(-)

diff --git a/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedNorthPole.java b/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedNorthPole.java
new file mode 100644
index 0000000..22b2ae9
--- /dev/null
+++ b/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedNorthPole.java
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+package org.apache.sis.internal.referencing.provider;
+
+import javax.xml.bind.annotation.XmlTransient;
+import org.opengis.util.FactoryException;
+import org.opengis.parameter.ParameterValueGroup;
+import org.opengis.parameter.ParameterDescriptor;
+import org.opengis.parameter.ParameterDescriptorGroup;
+import org.opengis.referencing.operation.MathTransform;
+import org.opengis.referencing.operation.MathTransformFactory;
+import org.apache.sis.referencing.operation.transform.RotatedPole;
+import org.apache.sis.metadata.iso.citation.Citations;
+import org.apache.sis.parameter.ParameterBuilder;
+import org.apache.sis.parameter.Parameters;
+import org.apache.sis.measure.Longitude;
+import org.apache.sis.measure.Latitude;
+import org.apache.sis.measure.Units;
+
+
+/**
+ * The provider for the NetCDF <cite>Rotated Latitude/Longitude</cite> coordinate operation.
+ * This is similar to the WMO Rotated Latitude/Longitude but rotating north pole instead of
+ * south pole.
+ *
+ * @author  Martin Desruisseaux (Geomatys)
+ * @version 1.2
+ *
+ * @see <a href="https://cfconventions.org/cf-conventions/cf-conventions.html#_rotated_pole">Rotated pole in CF-conversions</a>
+ *
+ * @since 1.2
+ * @module
+ */
+@XmlTransient
+public final class RotatedNorthPole extends AbstractProvider {
+    /**
+     * For cross-version compatibility.
+     */
+    private static final long serialVersionUID = 3485083285768740448L;
+
+    /**
+     * The operation parameter descriptor for the <cite>grid north pole latitude</cite> parameter value.
+     *
+     * <!-- Generated by ParameterNameTableGenerator -->
+     * <table class="sis">
+     *   <caption>Parameter names</caption>
+     *   <tr><td> NetCDF:  </td><td> grid_north_pole_latitude </td></tr>
+     * </table>
+     * <b>Notes:</b>
+     * <ul>
+     *   <li>No default value</li>
+     * </ul>
+     */
+    private static final ParameterDescriptor<Double> GRID_POLE_LATITUDE;
+
+    /**
+     * The operation parameter descriptor for the <cite>grid north pole longitude</cite> parameter value.
+     *
+     * <!-- Generated by ParameterNameTableGenerator -->
+     * <table class="sis">
+     *   <caption>Parameter names</caption>
+     *   <tr><td> NetCDF:  </td><td> grid_north_pole_longitude </td></tr>
+     * </table>
+     * <b>Notes:</b>
+     * <ul>
+     *   <li>No default value</li>
+     * </ul>
+     */
+    private static final ParameterDescriptor<Double> GRID_POLE_LONGITUDE;
+
+    /**
+     * The operation parameter descriptor for the <cite>north_pole_grid_longitude</cite> parameter value.
+     * This parameter is optional.
+     *
+     * <!-- Generated by ParameterNameTableGenerator -->
+     * <table class="sis">
+     *   <caption>Parameter names</caption>
+     *   <tr><td> NetCDF:  </td><td> north_pole_grid_longitude </td></tr>
+     * </table>
+     * <b>Notes:</b>
+     * <ul>
+     *   <li>Value domain: [-180.0 … 180.0]°</li>
+     *   <li>Optional</li>
+     * </ul>
+     */
+    private static final ParameterDescriptor<Double> GRID_POLE_ANGLE;
+
+    /**
+     * The group of all parameters expected by this coordinate operation.
+     */
+    public static final ParameterDescriptorGroup PARAMETERS;
+    static {
+        final ParameterBuilder builder = new ParameterBuilder().setCodeSpace(Citations.NETCDF, "NetCDF").setRequired(true);
+
+        GRID_POLE_LATITUDE = builder.addName("grid_north_pole_latitude")
+                .createBounded(Latitude.MIN_VALUE, Latitude.MAX_VALUE, Double.NaN, Units.DEGREE);
+
+        GRID_POLE_LONGITUDE = builder.addName("grid_north_pole_longitude")
+                .createBounded(Longitude.MIN_VALUE, Longitude.MAX_VALUE, Double.NaN, Units.DEGREE);
+
+        GRID_POLE_ANGLE = builder.setRequired(false).addName("north_pole_grid_longitude")
+                .createBounded(Longitude.MIN_VALUE, Longitude.MAX_VALUE, 0, Units.DEGREE);
+
+        PARAMETERS = builder.setRequired(true)
+                .addName("rotated_latitude_longitude")
+                .createGroup(GRID_POLE_LATITUDE,    // Note: `RotatedPole` implementation depends on this parameter order.
+                             GRID_POLE_LONGITUDE,
+                             GRID_POLE_ANGLE);
+    }
+
+    /**
+     * Constructs a new provider.
+     */
+    public RotatedNorthPole() {
+        super(2, 2, PARAMETERS);
+    }
+
+    /**
+     * Creates a coordinate operation from the specified group of parameter values.
+     *
+     * @param  factory     the factory to use for creating the transforms.
+     * @param  parameters  the group of parameter values.
+     * @return the coordinate operation created from the given parameter values.
+     * @throws FactoryException if the coordinate operation can not be created.
+     */
+    @Override
+    public MathTransform createMathTransform(final MathTransformFactory factory, final ParameterValueGroup parameters)
+            throws FactoryException
+    {
+        final Parameters p = Parameters.castOrWrap(parameters);
+        return RotatedPole.rotateNorthPole(factory,
+                p.getValue(GRID_POLE_LONGITUDE),
+                p.getValue(GRID_POLE_LATITUDE),
+                p.getValue(GRID_POLE_ANGLE));
+    }
+}
diff --git a/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedSouthPole.java b/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedSouthPole.java
new file mode 100644
index 0000000..734e4b2
--- /dev/null
+++ b/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/RotatedSouthPole.java
@@ -0,0 +1,154 @@
+/*
+ * 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.
+ */
+package org.apache.sis.internal.referencing.provider;
+
+import javax.xml.bind.annotation.XmlTransient;
+import org.opengis.util.FactoryException;
+import org.opengis.parameter.ParameterValueGroup;
+import org.opengis.parameter.ParameterDescriptor;
+import org.opengis.parameter.ParameterDescriptorGroup;
+import org.opengis.referencing.operation.MathTransform;
+import org.opengis.referencing.operation.MathTransformFactory;
+import org.apache.sis.referencing.operation.transform.RotatedPole;
+import org.apache.sis.metadata.iso.citation.Citations;
+import org.apache.sis.parameter.ParameterBuilder;
+import org.apache.sis.parameter.Parameters;
+import org.apache.sis.measure.Longitude;
+import org.apache.sis.measure.Latitude;
+import org.apache.sis.measure.Units;
+
+
+/**
+ * The provider for the WMO <cite>Rotated Latitude/Longitude</cite> coordinate operation.
+ * This is defined by the World Meteorological Organization (WMO) in GRIB2 template 3.1.
+ *
+ * @author  Martin Desruisseaux (Geomatys)
+ * @version 1.2
+ * @since   1.2
+ * @module
+ */
+@XmlTransient
+public final class RotatedSouthPole extends AbstractProvider {
+    /**
+     * For cross-version compatibility.
+     */
+    private static final long serialVersionUID = -5970630604222205521L;
+
+    /**
+     * The operation parameter descriptor for the <cite>grid south pole latitude</cite> parameter value.
+     * This is the geographic latitude (usually in degrees) of the southern pole of the coordinate system.
+     * The symbol used in GRIB2 template 3.1 is θ<sub>p</sub>.
+     *
+     * <!-- Generated by ParameterNameTableGenerator -->
+     * <table class="sis">
+     *   <caption>Parameter names</caption>
+     *   <tr><td> NetCDF:  </td><td> grid_south_pole_latitude </td></tr>
+     * </table>
+     * <b>Notes:</b>
+     * <ul>
+     *   <li>No default value</li>
+     * </ul>
+     */
+    private static final ParameterDescriptor<Double> GRID_POLE_LATITUDE;
+
+    /**
+     * The operation parameter descriptor for the <cite>grid south pole longitude</cite> parameter value.
+     * This is the geographic longitude (usually in degrees) of the southern pole of the coordinate system.
+     * The symbol used in GRIB2 template 3.1 is λ<sub>p</sub>.
+     *
+     * <!-- Generated by ParameterNameTableGenerator -->
+     * <table class="sis">
+     *   <caption>Parameter names</caption>
+     *   <tr><td> NetCDF:  </td><td> grid_south_pole_longitude </td></tr>
+     * </table>
+     * <b>Notes:</b>
+     * <ul>
+     *   <li>No default value</li>
+     * </ul>
+     */
+    private static final ParameterDescriptor<Double> GRID_POLE_LONGITUDE;
+
+    /**
+     * The operation parameter descriptor for the <cite>grid_south_pole_angle</cite> parameter value (optional).
+     * This is the angle of rotation about the new polar axis (measured clockwise when looking from the southern
+     * to the northern pole) of the coordinate system, assuming the new axis to have been obtained by first
+     * rotating the sphere through λ<sub>p</sub> about the geographic polar axis, and then rotating through
+     * (90° + θ<sub>p</sub>) degrees so that the southern pole moved along the (previously rotated) Greenwich meridian.
+     *
+     * <!-- Generated by ParameterNameTableGenerator -->
+     * <table class="sis">
+     *   <caption>Parameter names</caption>
+     *   <tr><td> NetCDF:  </td><td> grid_south_pole_angle </td></tr>
+     * </table>
+     * <b>Notes:</b>
+     * <ul>
+     *   <li>Value domain: [-180.0 … 180.0]°</li>
+     *   <li>Optional</li>
+     * </ul>
+     */
+    private static final ParameterDescriptor<Double> GRID_POLE_ANGLE;
+
+    /**
+     * The group of all parameters expected by this coordinate operation.
+     */
+    public static final ParameterDescriptorGroup PARAMETERS;
+    static {
+        final ParameterBuilder builder = new ParameterBuilder().setCodeSpace(Citations.NETCDF, "NetCDF").setRequired(true);
+
+        GRID_POLE_LATITUDE = builder.addName("grid_south_pole_latitude")
+                .createBounded(Latitude.MIN_VALUE, Latitude.MAX_VALUE, Double.NaN, Units.DEGREE);
+
+        GRID_POLE_LONGITUDE = builder.addName("grid_south_pole_longitude")
+                .createBounded(Longitude.MIN_VALUE, Longitude.MAX_VALUE, Double.NaN, Units.DEGREE);
+
+        GRID_POLE_ANGLE = builder.setRequired(false).addName("grid_south_pole_angle")
+                .createBounded(Longitude.MIN_VALUE, Longitude.MAX_VALUE, 0, Units.DEGREE);
+
+        PARAMETERS = builder.setRequired(true)
+                .addName(Citations.WMO, "Rotated Latitude/longitude")
+                .addName("rotated_latlon_grib")
+                .createGroup(GRID_POLE_LATITUDE,    // Note: `RotatedPole` implementation depends on this parameter order.
+                             GRID_POLE_LONGITUDE,
+                             GRID_POLE_ANGLE);
+    }
+
+    /**
+     * Constructs a new provider.
+     */
+    public RotatedSouthPole() {
+        super(2, 2, PARAMETERS);
+    }
+
+    /**
+     * Creates a coordinate operation from the specified group of parameter values.
+     *
+     * @param  factory     the factory to use for creating the transforms.
+     * @param  parameters  the group of parameter values.
+     * @return the coordinate operation created from the given parameter values.
+     * @throws FactoryException if the coordinate operation can not be created.
+     */
+    @Override
+    public MathTransform createMathTransform(final MathTransformFactory factory, final ParameterValueGroup parameters)
+            throws FactoryException
+    {
+        final Parameters p = Parameters.castOrWrap(parameters);
+        return RotatedPole.rotateSouthPole(factory,
+                p.getValue(GRID_POLE_LONGITUDE),
+                p.getValue(GRID_POLE_LATITUDE),
+                p.getValue(GRID_POLE_ANGLE));
+    }
+}
diff --git a/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/package-info.java b/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/package-info.java
index ecc9b95..466e271 100644
--- a/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/package-info.java
+++ b/core/sis-referencing/src/main/java/org/apache/sis/internal/referencing/provider/package-info.java
@@ -22,7 +22,7 @@
  *
  * @author  Martin Desruisseaux (Geomatys)
  * @author  Matthieu Bastianelli (Geomatys)
- * @version 1.1
+ * @version 1.2
  *
  * @see org.apache.sis.referencing.operation.transform.MathTransformProvider
  *
diff --git a/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/RotatedPole.java b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/RotatedPole.java
new file mode 100644
index 0000000..5b19036
--- /dev/null
+++ b/core/sis-referencing/src/main/java/org/apache/sis/referencing/operation/transform/RotatedPole.java
@@ -0,0 +1,357 @@
+/*
+ * 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.
+ */
+package org.apache.sis.referencing.operation.transform;
+
+import java.util.List;
+import java.util.Collections;
+import java.io.Serializable;
+import org.opengis.util.FactoryException;
+import org.opengis.referencing.operation.Matrix;
+import org.opengis.referencing.operation.MathTransform;
+import org.opengis.referencing.operation.MathTransform2D;
+import org.opengis.referencing.operation.MathTransformFactory;
+import org.opengis.referencing.operation.TransformException;
+import org.opengis.parameter.GeneralParameterValue;
+import org.opengis.parameter.ParameterValue;
+import org.opengis.parameter.ParameterValueGroup;
+import org.opengis.parameter.ParameterDescriptor;
+import org.opengis.parameter.ParameterDescriptorGroup;
+import org.apache.sis.parameter.DefaultParameterDescriptorGroup;
+import org.apache.sis.parameter.Parameters;
+import org.apache.sis.internal.referencing.provider.RotatedNorthPole;
+import org.apache.sis.internal.referencing.provider.RotatedSouthPole;
+import org.apache.sis.internal.referencing.Formulas;
+import org.apache.sis.internal.util.Numerics;
+import org.apache.sis.internal.util.Constants;
+import org.apache.sis.metadata.iso.citation.Citations;
+import org.apache.sis.referencing.ImmutableIdentifier;
+import org.apache.sis.util.ComparisonMode;
+import org.apache.sis.util.Debug;
+
+import static java.lang.Math.*;
+
+
+/**
+ * Computes latitudes and longitudes on a sphere where the south pole has been moved to given geographic coordinates.
+ * The parameter values of this transform use the conventions defined in template 3.1 of GRIB2 format published by the
+ * <a href="https://www.wmo.int/">World Meteorological Organization</a> (WMO):
+ *
+ * <ol>
+ *   <li><b>λ<sub>p</sub>:</b> geographic longitude in degrees of the southern pole of the coordinate system.</li>
+ *   <li><b>θ<sub>p</sub>:</b> geographic  latitude in degrees of the southern pole of the coordinate system.</li>
+ *   <li>Angle of rotation in degrees about the new polar axis measured clockwise when looking from the southern
+ *       to the northern pole.</li>
+ * </ol>
+ *
+ * The rotations are applied by first rotating the sphere through λ<sub>p</sub> about the geographic polar axis,
+ * and then rotating through (θ<sub>p</sub> − (−90°)) degrees so that the southern pole moved along the
+ * (previously rotated) Greenwich meridian.
+ *
+ * <p>Source and target axis order is (<var>longitude</var>, <var>latitude</var>).
+ * This is the usual axis order used by Apache SIS for <em>internal</em> calculations.
+ * If a different axis order is desired (for example for showing coordinates to the user),
+ * an affine transform can be concatenated to this transform.</p>
+ *
+ * @author  Martin Desruisseaux (Geomatys)
+ * @version 1.2
+ * @since   1.2
+ * @module
+ */
+public class RotatedPole extends AbstractMathTransform2D implements Serializable {
+    /**
+     * For cross-version compatibility.
+     */
+    private static final long serialVersionUID = -8355693495724373931L;
+
+    /**
+     * The parameters used for creating this transform.
+     * They are used for formatting <cite>Well Known Text</cite> (WKT).
+     *
+     * @see #getContextualParameters()
+     */
+    private final ContextualParameters context;
+
+    /**
+     * Sine and cosine of the geographic latitude of the southern pole of the coordinate system.
+     * The rotation angle to apply is (θ<sub>p</sub> − (−90°)) degrees for the south pole (−90°),
+     * but we use the following trigonometric identities:
+     *
+     * <p>For the south pole:</p>
+     * <ul>
+     *   <li>sin(θ + 90°) =  cos(θ)</li>
+     *   <li>cos(θ + 90°) = −sin(θ)</li>
+     * </ul>
+     *
+     * <p>For the north pole:</p>
+     * <ul>
+     *   <li>sin(θ − 90°) = −cos(θ)</li>
+     *   <li>cos(θ − 90°) =  sin(θ)</li>
+     * </ul>
+     *
+     * By convention those fields contain the sine and cosine for the south pole case,
+     * and values with opposite sign for the north pole case.
+     */
+    private final double sinθp, cosθp;
+
+    /**
+     * The inverse of this operation, computed when first needed.
+     *
+     * @see #inverse()
+     */
+    private MathTransform2D inverse;
+
+    /**
+     * Creates the inverse of the given forward operation.
+     *
+     * @see #inverse()
+     */
+    private RotatedPole(final RotatedPole forward) {
+        context =  forward.context.inverse(forward.context.getDescriptor(), RotatedPole::inverseParameter);
+        sinθp   =  forward.sinθp;
+        cosθp   = -forward.cosθp;
+        inverse =  forward;
+    }
+
+    /**
+     * Computes the value of the given parameter for the inverse operation.
+     * This method is invoked for each parameter.
+     *
+     * @param  forward  the forward operation.
+     * @param  target   parameter to initialize.
+     * @return whether to accept the parameter (always {@code true}).
+     */
+    private static boolean inverseParameter(final Parameters forward, final ParameterValue<?> target) {
+        final ParameterDescriptor<?> descriptor = target.getDescriptor();
+        final List<GeneralParameterValue> values = forward.values();
+        for (int i = values.size(); --i >= 0;) {
+            if (descriptor.equals(values.get(i).getDescriptor())) {
+                if (i != 0) {
+                    /*
+                     * For assigning a value to the "grid_south_pole_longitude" parameter at index 1,
+                     * we derive the value from the "grid_south_pole_angle" parameter at index 2.
+                     * And conversely.
+                     */
+                    i = 3 - i;
+                }
+                double value = -((Number) ((ParameterValue<?>) values.get(i)).getValue()).doubleValue();
+                if (i == 0) value = Math.IEEEremainder(value + 180, 360);
+                target.setValue(value);
+                return true;
+            }
+        }
+        return false;       // Should never happen.
+    }
+
+    /**
+     * Creates the non-linear part of a rotated pole operation.
+     * This transform does not include the conversion between degrees and radians and the longitude rotations.
+     * For a complete transform, use one of the static factory methods.
+     *
+     * @param  south  {@code true} for a south pole rotation, or {@code false} for a north pole rotation.
+     * @param  λp     geographic longitude in degrees of the southern pole of the coordinate system.
+     * @param  θp     geographic latitude in degrees of the southern pole of the coordinate system.
+     * @param  pa     angle of rotation in degrees about the new polar axis measured clockwise when
+     *                looking from the southern to the northern pole.
+     */
+    protected RotatedPole(final boolean south, double λp, double θp, double pa) {
+        context = new ContextualParameters(
+                south ? RotatedSouthPole.PARAMETERS
+                      : RotatedNorthPole.PARAMETERS, 2, 2);
+        setValue(0, θp);        // grid_south_pole_latitude   or  grid_north_pole_latitude
+        setValue(1, λp);        // grid_south_pole_longitude  or  grid_north_pole_longitude
+        setValue(2, pa);        // grid_south_pole_angle      or  north_pole_grid_longitude
+        final double θ = toRadians(θp);
+        final double sign = south ? 1 : -1;
+        sinθp = sin(θ) * sign;
+        cosθp = cos(θ) * sign;
+        context.normalizeGeographicInputs(λp);
+        context.denormalizeGeographicOutputs(-pa);
+    }
+
+    /**
+     * Sets the value of the parameter at the given index.
+     * In the rotated south pole case, parameter 0 to 2 (inclusive) are:
+     * {@code "grid_south_pole_latitude"},
+     * {@code "grid_south_pole_longitude"} and
+     * {@code "grid_south_pole_angle"} in that order.
+     */
+    private void setValue(final int index, final double value) {
+        final ParameterDescriptor<?> p = (ParameterDescriptor<?>) context.getDescriptor().descriptors().get(index);
+        context.parameter(p.getName().getCode()).setValue(value);
+    }
+
+    /**
+     * Creates a new rotated south pole operation.
+     *
+     * @param  factory  the factory to use for creating the transform.
+     * @param  λp       geographic longitude in degrees of the southern pole of the coordinate system.
+     * @param  θp       geographic latitude in degrees of the southern pole of the coordinate system.
+     * @param  pa       angle of rotation in degrees about the new polar axis measured clockwise when
+     *                  looking from the southern to the northern pole.
+     * @return the conversion doing a south pole rotation.
+     * @throws FactoryException if an error occurred while creating a transform.
+     */
+    public static MathTransform rotateSouthPole(final MathTransformFactory factory,
+            final double λp, final double θp, final double pa) throws FactoryException
+    {
+        final RotatedPole kernel = new RotatedPole(true, λp, θp, pa);
+        return kernel.context.completeTransform(factory, kernel);
+    }
+
+    /**
+     * Creates a new rotated north pole operation.
+     *
+     * @param  factory  the factory to use for creating the transform.
+     * @param  λp       geographic longitude in degrees of the northern pole of the coordinate system.
+     * @param  θp       geographic latitude in degrees of the northern pole of the coordinate system.
+     * @param  pa       angle of rotation in degrees about the new polar axis measured clockwise when
+     *                  looking from the northern to the southern pole.
+     * @return the conversion doing a north pole rotation.
+     * @throws FactoryException if an error occurred while creating a transform.
+     */
+    public static MathTransform rotateNorthPole(final MathTransformFactory factory,
+            final double λp, final double θp, final double pa) throws FactoryException
+    {
+        final RotatedPole kernel = new RotatedPole(false, λp, θp, pa);
+        return kernel.context.completeTransform(factory, kernel);
+    }
+
+    /**
+     * Returns a description of the parameters of this transform. The group of parameters contains only the grid
+     * (north or south) pole latitude. It does not contain the grid pole longitude or the grid angle of rotation
+     * because those parameters are handled by affine transforms pre- or post-concatenated to this transform.
+     *
+     * @return the parameter descriptors for this math transform.
+     */
+    @Debug
+    @Override
+    public ParameterDescriptorGroup getParameterDescriptors() {
+        // We assume that it is not worth to cache this descriptor.
+        ImmutableIdentifier name = new ImmutableIdentifier(Citations.SIS, Constants.SIS, "Rotated Latitude/longitude (radians domain)");
+        return new DefaultParameterDescriptorGroup(Collections.singletonMap(ParameterDescriptorGroup.NAME_KEY, name),
+                1, 1, (ParameterDescriptor<?>) context.getDescriptor().descriptors().get(0));
+    }
+
+    /**
+     * Returns a copy of the parameter values of this transform.
+     * The group contains the values of the parameters described by {@link #getParameterDescriptors()}.
+     * This method is mostly for {@linkplain org.apache.sis.io.wkt.Convention#INTERNAL debugging purposes};
+     * most GIS applications will instead be interested in the {@linkplain #getContextualParameters()
+     * contextual parameters} instead.
+     *
+     * @return the parameter values for this math transform.
+     */
+    @Debug
+    @Override
+    public ParameterValueGroup getParameterValues() {
+        final ParameterValueGroup values = getParameterDescriptors().createValue();
+        values.values().add(context.values().get(0));           // First parameter is grid pole latitude.
+        return values;
+    }
+
+    /**
+     * Returns the parameters used for creating the complete operation. The returned group contains not only
+     * the grid pole latitude (which is handled by this transform), but also the grid pole longitude and the
+     * grid angle of rotation (which are handled by affine transforms before or after this transform).
+     *
+     * @return the parameter values for the sequence of <cite>normalize</cite> →
+     *         {@code this} → <cite>denormalize</cite> transforms.
+     */
+    @Override
+    protected ContextualParameters getContextualParameters() {
+        return context;
+    }
+
+    /**
+     * Transforms a single coordinate point in an array,
+     * and optionally computes the transform derivative at that location.
+     */
+    @Override
+    public Matrix transform(final double[] srcPts, final int srcOff,
+                            final double[] dstPts, final int dstOff,
+                            final boolean derivate) throws TransformException
+    {
+        /*
+         * Convert latitude and longitude coordinates to (x,y,z) Cartesian coordinates on a sphere of radius 1.
+         * Note that the rotation around the Z axis has been performed in geographic coordinates by the affine
+         * transform pre-concatenated to this transform, simply by subtracting λp from the longitude value.
+         * This is simpler than performing the rotation in Cartesian coordinates.
+         */
+        double λ    = srcPts[srcOff];
+        double φ    = srcPts[srcOff+1];
+        double z    = sin(φ);
+        double cosφ = cos(φ);
+        double y    = sin(λ) * cosφ;
+        double x    = cos(λ) * cosφ;
+        /*
+         * Apply the rotation around Y axis (so the y value stay unchanged)
+         * and convert back to spherical coordinates.
+         */
+        double xr =  cosθp * z - sinθp * x;
+        double zr = -cosθp * x - sinθp * z;
+        double R  = sqrt(xr*xr + y*y);          // The slower hypot(…) is not needed because values are close to 1.
+        dstPts[dstOff]   = atan2(y, xr);
+        dstPts[dstOff+1] = atan2(zr, R);
+        if (!derivate) {
+            return null;
+        }
+        throw new TransformException();         // TODO
+    }
+
+    /**
+     * Returns the inverse transform of this object.
+     *
+     * @return the inverse of this transform.
+     */
+    @Override
+    public synchronized MathTransform2D inverse() {
+        if (inverse == null) {
+            inverse = new RotatedPole(this);
+        }
+        return inverse;
+    }
+
+    /**
+     * Compares the specified object with this math transform for equality.
+     *
+     * @param  object  the object to compare with this transform.
+     * @param  mode    the strictness level of the comparison.
+     * @return {@code true} if the given object is considered equals to this math transform.
+     */
+    @Override
+    public boolean equals(final Object object, final ComparisonMode mode) {
+        if (super.equals(object, mode)) {
+            final RotatedPole other = (RotatedPole) object;
+            if (mode.isApproximate()) {
+                return Numerics.epsilonEqual(sinθp, other.sinθp, Formulas.ANGULAR_TOLERANCE * (PI/180)) &&
+                       Numerics.epsilonEqual(cosθp, other.cosθp, Formulas.ANGULAR_TOLERANCE * (PI/180));
+            } else {
+                return Numerics.equals(sinθp, other.sinθp) &&
+                       Numerics.equals(cosθp, other.cosθp);
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Computes a hash value for this transform. This method is invoked by {@link #hashCode()} when first needed.
+     */
+    @Override
+    protected int computeHashCode() {
+        return super.computeHashCode() + Double.hashCode(cosθp) + Double.hashCode(sinθp);
+    }
+}
diff --git a/core/sis-referencing/src/main/resources/META-INF/services/org.opengis.referencing.operation.OperationMethod b/core/sis-referencing/src/main/resources/META-INF/services/org.opengis.referencing.operation.OperationMethod
index 429a260..2526f0e 100644
--- a/core/sis-referencing/src/main/resources/META-INF/services/org.opengis.referencing.operation.OperationMethod
+++ b/core/sis-referencing/src/main/resources/META-INF/services/org.opengis.referencing.operation.OperationMethod
@@ -63,6 +63,8 @@ org.apache.sis.internal.referencing.provider.ZonedTransverseMercator
 org.apache.sis.internal.referencing.provider.Sinusoidal
 org.apache.sis.internal.referencing.provider.Polyconic
 org.apache.sis.internal.referencing.provider.Mollweide
+org.apache.sis.internal.referencing.provider.RotatedSouthPole
+org.apache.sis.internal.referencing.provider.RotatedNorthPole
 org.apache.sis.internal.referencing.provider.NTv2
 org.apache.sis.internal.referencing.provider.NTv1
 org.apache.sis.internal.referencing.provider.NADCON
diff --git a/core/sis-referencing/src/test/java/org/apache/sis/internal/referencing/provider/ProvidersTest.java b/core/sis-referencing/src/test/java/org/apache/sis/internal/referencing/provider/ProvidersTest.java
index e3b4513..21351a0 100644
--- a/core/sis-referencing/src/test/java/org/apache/sis/internal/referencing/provider/ProvidersTest.java
+++ b/core/sis-referencing/src/test/java/org/apache/sis/internal/referencing/provider/ProvidersTest.java
@@ -36,7 +36,7 @@ import static org.junit.Assert.*;
  * Tests {@link Providers} and some consistency rules of all providers defined in this package.
  *
  * @author  Martin Desruisseaux (Geomatys)
- * @version 1.1
+ * @version 1.2
  * @since   0.6
  * @module
  */
@@ -113,6 +113,8 @@ public final strictfp class ProvidersTest extends TestCase {
             Sinusoidal.class,
             Polyconic.class,
             Mollweide.class,
+            RotatedSouthPole.class,
+            RotatedNorthPole.class,
             NTv2.class,
             NTv1.class,
             NADCON.class,
diff --git a/core/sis-referencing/src/test/java/org/apache/sis/referencing/operation/transform/RotatedPoleTest.java b/core/sis-referencing/src/test/java/org/apache/sis/referencing/operation/transform/RotatedPoleTest.java
new file mode 100644
index 0000000..8cca70c
--- /dev/null
+++ b/core/sis-referencing/src/test/java/org/apache/sis/referencing/operation/transform/RotatedPoleTest.java
@@ -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.
+ */
+package org.apache.sis.referencing.operation.transform;
+
+import org.junit.Test;
+import org.opengis.util.FactoryException;
+import org.opengis.parameter.ParameterValueGroup;
+import org.opengis.referencing.operation.MathTransformFactory;
+import org.opengis.referencing.operation.TransformException;
+import org.apache.sis.internal.referencing.Formulas;
+import org.apache.sis.parameter.Parameterized;
+import org.apache.sis.test.DependsOnMethod;
+
+
+/**
+ * Tests {@link RotatedPole}.
+ *
+ * @author  Martin Desruisseaux (Geomatys)
+ * @version 1.2
+ * @since   1.2
+ * @module
+ */
+public final strictfp class RotatedPoleTest extends MathTransformTestCase {
+    /**
+     * Returns the transform factory to use for testing purpose.
+     * This mock supports only the "affine" and "concatenate" operations.
+     */
+    private static MathTransformFactory factory() {
+        return new MathTransformFactoryMock(null);
+    }
+
+    /**
+     * Creates a new transform which should be the inverse of current transform according the
+     * parameters declared in {@link RotatedPole#context}. Those parameters may be wrong even
+     * if the coordinates transformed by {@code transform.inverse()} are corrects because the
+     * parameters are only for WKT formatting (they are not actually used for transformation,
+     * unless we force their use as done in this method).
+     */
+    private void inverseSouthPoleTransform() throws FactoryException, TransformException {
+        final ParameterValueGroup pg = ((Parameterized) transform.inverse()).getParameterValues();
+        transform = RotatedPole.rotateSouthPole(factory(),
+                pg.parameter("grid_south_pole_longitude").doubleValue(),
+                pg.parameter("grid_south_pole_latitude") .doubleValue(),
+                pg.parameter("grid_south_pole_angle")    .doubleValue());
+
+    }
+
+    /**
+     * Tests a rotation of south pole with the new pole on Greenwich.
+     * The {@link ucar.unidata.geoloc.LatLonPoint} class has been used
+     * as a reference implementation for computing the expected values.
+     *
+     * @throws FactoryException if the transform can not be created.
+     * @throws TransformException if an error occurred while transforming a point.
+     */
+    @Test
+    public void testRotateSouthPoleOnGreenwich() throws FactoryException, TransformException {
+        transform = RotatedPole.rotateSouthPole(factory(), 0, -60, 0);
+        tolerance = Formulas.ANGULAR_TOLERANCE;
+        isDerivativeSupported = false;
+        final double[] coordinates = {      // (λ,φ) coordinates to convert.
+              0, -51,
+             20, -51,
+            100, -61
+        };
+        final double[] expected = {         // (λ,φ) coordinates after conversion.
+              0.000000000, -81.000000000,
+             60.140453893, -75.629715301,
+            136.900518716, -45.671868261
+        };
+        verifyTransform(coordinates, expected);
+        inverseSouthPoleTransform();
+        verifyTransform(expected, coordinates);
+    }
+
+    /**
+     * Tests a rotation of south pole with the pole on arbitrary meridian.
+     * The {@link ucar.unidata.geoloc.LatLonPoint} class has been used as
+     * a reference implementation for computing the expected values.
+     *
+     * @throws FactoryException if the transform can not be created.
+     * @throws TransformException if an error occurred while transforming a point.
+     */
+    @Test
+    @DependsOnMethod("testRotateSouthPoleOnGreenwich")
+    public void testRotateSouthPoleWithAngle() throws FactoryException, TransformException {
+        transform = RotatedPole.rotateSouthPole(factory(), 20, -50, 10);
+        tolerance = Formulas.ANGULAR_TOLERANCE;
+        isDerivativeSupported = false;
+        final double[] coordinates = {      // (λ,φ) coordinates to convert.
+             20, -51,
+             80, -44,
+            -30, -89
+        };
+        final double[] expected = {         // (λ,φ) coordinates after conversion.
+             170.000000000, -89.000000000,
+              95.348788748, -49.758697265,
+            -188.792151374, -50.636582758
+        };
+        verifyTransform(coordinates, expected);
+        inverseSouthPoleTransform();
+        verifyTransform(expected, coordinates);
+    }
+}
diff --git a/core/sis-referencing/src/test/java/org/apache/sis/test/suite/ReferencingTestSuite.java b/core/sis-referencing/src/test/java/org/apache/sis/test/suite/ReferencingTestSuite.java
index 42130b8..412d597 100644
--- a/core/sis-referencing/src/test/java/org/apache/sis/test/suite/ReferencingTestSuite.java
+++ b/core/sis-referencing/src/test/java/org/apache/sis/test/suite/ReferencingTestSuite.java
@@ -142,6 +142,7 @@ import org.junit.BeforeClass;
     org.apache.sis.referencing.operation.transform.EllipsoidToCentricTransformTest.class,
     org.apache.sis.referencing.operation.transform.MolodenskyTransformTest.class,
     org.apache.sis.referencing.operation.transform.AbridgedMolodenskyTransformTest.class,
+    org.apache.sis.referencing.operation.transform.RotatedPoleTest.class,
     org.apache.sis.referencing.operation.transform.SphericalToCartesianTest.class,
     org.apache.sis.referencing.operation.transform.CartesianToSphericalTest.class,
     org.apache.sis.referencing.operation.transform.PolarToCartesianTest.class,