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

jiayu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/sedona.git


The following commit(s) were added to refs/heads/master by this push:
     new fcf57c0c [SEDONA-251] Add raster type to Sedona (#773)
fcf57c0c is described below

commit fcf57c0c55b22e4f26288af124dccf8a59b82475
Author: Martin Andersson <[email protected]>
AuthorDate: Thu Feb 23 11:10:08 2023 +0100

    [SEDONA-251] Add raster type to Sedona (#773)
---
 common/pom.xml                                     |   8 ++
 .../apache/sedona/common/raster/Constructors.java  |  38 +++++++
 .../org/apache/sedona/common/raster/Functions.java |  91 +++++++++++++++++
 .../org/apache/sedona/common/raster/Serde.java     |  61 ++++++++++++
 .../sedona/common/raster/ConstructorsTest.java     |  54 ++++++++++
 .../apache/sedona/common/raster/FunctionsTest.java |  90 +++++++++++++++++
 .../sedona/common/raster/RasterTestBase.java       |  56 +++++++++++
 .../org/apache/sedona/common/raster/SerdeTest.java |  40 ++++++++
 core/src/test/resources/raster_asc/test1.asc       |   7 ++
 docs/api/sql/Raster-loader.md                      |  33 +++++++
 docs/api/sql/Raster-operators.md                   | 110 +++++++++++++++++++++
 pom.xml                                            |   6 ++
 spark-shaded/pom.xml                               |   4 +
 sql/pom.xml                                        |   4 +
 .../scala/org/apache/sedona/sql/UDF/Catalog.scala  |   8 +-
 .../spark/sql/sedona_sql/UDT/RasterUDT.scala       |  22 +++++
 .../sql/sedona_sql/UDT/UdtRegistratorWrapper.scala |  13 +++
 .../expressions/raster/Constructors.scala          |  76 ++++++++++++++
 .../sedona_sql/expressions/raster/Functions.scala  | 107 +++++++++++++++++++-
 .../raster/implicits.scala}                        |  26 +++--
 .../org/apache/sedona/sql/rasteralgebraTest.scala  |  78 ++++++++++++++-
 21 files changed, 920 insertions(+), 12 deletions(-)

diff --git a/common/pom.xml b/common/pom.xml
index eb7bc188..7344067b 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -49,6 +49,14 @@
             <groupId>org.geotools</groupId>
             <artifactId>gt-epsg-hsql</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.geotools</groupId>
+            <artifactId>gt-geotiff</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.geotools</groupId>
+            <artifactId>gt-arcgrid</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.locationtech.jts</groupId>
             <artifactId>jts-core</artifactId>
diff --git 
a/common/src/main/java/org/apache/sedona/common/raster/Constructors.java 
b/common/src/main/java/org/apache/sedona/common/raster/Constructors.java
new file mode 100644
index 00000000..e39a80b7
--- /dev/null
+++ b/common/src/main/java/org/apache/sedona/common/raster/Constructors.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.geotools.coverage.grid.GridCoverage2D;
+import org.geotools.gce.arcgrid.ArcGridReader;
+import org.geotools.gce.geotiff.GeoTiffReader;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+public class Constructors {
+
+    public static GridCoverage2D fromArcInfoAsciiGrid(byte[] bytes) throws 
IOException {
+        try (ByteArrayInputStream inputStream = new 
ByteArrayInputStream(bytes)) {
+            ArcGridReader reader = new ArcGridReader(inputStream);
+            return reader.read(null);
+        }
+    }
+
+    public static GridCoverage2D fromGeoTiff(byte[] bytes) throws IOException {
+        try (ByteArrayInputStream inputStream = new 
ByteArrayInputStream(bytes)) {
+            GeoTiffReader geoTiffReader = new GeoTiffReader(inputStream);
+            return geoTiffReader.read(null);
+        }
+    }
+}
diff --git 
a/common/src/main/java/org/apache/sedona/common/raster/Functions.java 
b/common/src/main/java/org/apache/sedona/common/raster/Functions.java
new file mode 100644
index 00000000..489bd1d7
--- /dev/null
+++ b/common/src/main/java/org/apache/sedona/common/raster/Functions.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.geotools.coverage.grid.GridCoordinates2D;
+import org.geotools.coverage.grid.GridCoverage2D;
+import org.geotools.coverage.grid.GridGeometry2D;
+import org.geotools.geometry.DirectPosition2D;
+import org.geotools.geometry.Envelope2D;
+import org.locationtech.jts.geom.*;
+import org.opengis.referencing.operation.TransformException;
+
+import java.awt.image.Raster;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.DoublePredicate;
+import java.util.stream.Collectors;
+import java.util.stream.DoubleStream;
+
+public class Functions {
+
+    public static Geometry envelope(GridCoverage2D raster) {
+        Envelope2D envelope2D = raster.getEnvelope2D();
+
+        Envelope envelope = new Envelope(envelope2D.getMinX(), 
envelope2D.getMaxX(), envelope2D.getMinY(), envelope2D.getMaxY());
+        return new GeometryFactory().toGeometry(envelope);
+    }
+
+    public static int numBands(GridCoverage2D raster) {
+        return raster.getNumSampleDimensions();
+    }
+
+    public static Double value(GridCoverage2D rasterGeom, Geometry geometry, 
int band) throws TransformException {
+        return values(rasterGeom, Collections.singletonList(geometry), 
band).get(0);
+    }
+
+    public static List<Double> values(GridCoverage2D rasterGeom, 
List<Geometry> geometries, int band) throws TransformException {
+        int numBands = rasterGeom.getNumSampleDimensions();
+        if (band < 1 || band > numBands) {
+            // Invalid band index. Return nulls.
+            return geometries.stream().map(geom -> (Double) 
null).collect(Collectors.toList());
+        }
+        Raster raster = rasterGeom.getRenderedImage().getData();
+        GridGeometry2D gridGeometry = rasterGeom.getGridGeometry();
+        double[] noDataValues = rasterGeom.getSampleDimension(band - 
1).getNoDataValues();
+        DoublePredicate isNoData = d -> noDataValues != null && 
DoubleStream.of(noDataValues).anyMatch(noDataValue -> 
Double.compare(noDataValue, d) == 0);
+        double[] pixelBuffer = new double[numBands];
+
+        List<Double> result = new ArrayList<>(geometries.size());
+        for (Geometry geom : geometries) {
+            if (geom == null) {
+                result.add(null);
+            } else {
+                Point point = ensurePoint(geom);
+                DirectPosition2D directPosition2D = new 
DirectPosition2D(point.getX(), point.getY());
+                GridCoordinates2D gridCoordinates2D = 
gridGeometry.worldToGrid(directPosition2D);
+                try {
+                    double pixel = raster.getPixel(gridCoordinates2D.x, 
gridCoordinates2D.y, pixelBuffer)[band - 1];
+                    if (isNoData.test(pixel)) {
+                        result.add(null);
+                    } else {
+                        result.add(pixel);
+                    }
+                } catch (ArrayIndexOutOfBoundsException exc) {
+                    // Points outside the extent should return null
+                    result.add(null);
+                }
+            }
+        }
+        return result;
+    }
+
+    private static Point ensurePoint(Geometry geometry) {
+        if (geometry instanceof Point) {
+            return (Point) geometry;
+        }
+        throw new IllegalArgumentException("Attempting to get the value of a 
pixel with a non-point geometry.");
+    }
+}
diff --git a/common/src/main/java/org/apache/sedona/common/raster/Serde.java 
b/common/src/main/java/org/apache/sedona/common/raster/Serde.java
new file mode 100644
index 00000000..1762a543
--- /dev/null
+++ b/common/src/main/java/org/apache/sedona/common/raster/Serde.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.geotools.coverage.grid.GridCoverage2D;
+import org.geotools.coverage.grid.GridCoverageFactory;
+
+import javax.media.jai.remote.SerializableRenderedImage;
+import java.awt.image.RenderedImage;
+import java.io.*;
+
+public class Serde {
+
+    public static byte[] serialize(GridCoverage2D raster) throws IOException {
+        // GridCoverage2D created by GridCoverage2DReaders contain references 
that are not serializable.
+        // Wrap the RenderedImage in SerializableRenderedImage to make it 
serializable.
+        if (!(raster.getRenderedImage() instanceof SerializableRenderedImage)) 
{
+            RenderedImage renderedImage = new SerializableRenderedImage(
+                    raster.getRenderedImage(),
+                    false,
+                    null,
+                    "gzip",
+                    null,
+                    null
+            );
+            raster = new GridCoverageFactory().create(
+                    raster.getName(),
+                    renderedImage,
+                    raster.getGridGeometry(),
+                    raster.getSampleDimensions(),
+                    null,
+                    raster.getProperties()
+            );
+        }
+        try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) {
+            try (ObjectOutputStream oos = new ObjectOutputStream(bos)) {
+                oos.writeObject(raster);
+                return bos.toByteArray();
+            }
+        }
+    }
+
+    public static GridCoverage2D deserialize(byte[] bytes) throws IOException, 
ClassNotFoundException {
+        try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) {
+            try (ObjectInputStream ois = new ObjectInputStream(bis)) {
+                return (GridCoverage2D) ois.readObject();
+            }
+        }
+    }
+}
diff --git 
a/common/src/test/java/org/apache/sedona/common/raster/ConstructorsTest.java 
b/common/src/test/java/org/apache/sedona/common/raster/ConstructorsTest.java
new file mode 100644
index 00000000..69aa085b
--- /dev/null
+++ b/common/src/test/java/org/apache/sedona/common/raster/ConstructorsTest.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.geotools.coverage.grid.GridCoverage2D;
+import org.junit.Test;
+import org.locationtech.jts.geom.Geometry;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import static org.junit.Assert.*;
+
+public class ConstructorsTest extends RasterTestBase {
+
+    @Test
+    public void fromArcInfoAsciiGrid() throws IOException {
+        GridCoverage2D gridCoverage2D = 
Constructors.fromArcInfoAsciiGrid(arc.getBytes(StandardCharsets.UTF_8));
+
+        Geometry envelope = Functions.envelope(gridCoverage2D);
+        assertEquals(3600, envelope.getArea(), 0.1);
+        assertEquals(378922d + 30, envelope.getCentroid().getX(), 0.1);
+        assertEquals(4072345d + 30, envelope.getCentroid().getY(), 0.1);
+        assertEquals(2, gridCoverage2D.getRenderedImage().getTileHeight());
+        assertEquals(2, gridCoverage2D.getRenderedImage().getTileWidth());
+        assertEquals(0d, 
gridCoverage2D.getSampleDimension(0).getNoDataValues()[0], 0.1);
+        assertEquals(3d, 
gridCoverage2D.getRenderedImage().getData().getPixel(1, 1, (double[])null)[0], 
0.1);
+    }
+
+    @Test
+    public void fromGeoTiff() throws IOException {
+        GridCoverage2D gridCoverage2D = Constructors.fromGeoTiff(geoTiff);
+
+        Geometry envelope = Functions.envelope(gridCoverage2D);
+        assertEquals(100, envelope.getArea(), 0.1);
+        assertEquals(5, envelope.getCentroid().getX(), 0.1);
+        assertEquals(5, envelope.getCentroid().getY(), 0.1);
+        assertEquals(10, gridCoverage2D.getRenderedImage().getTileHeight());
+        assertEquals(10, gridCoverage2D.getRenderedImage().getTileWidth());
+        assertEquals(10d, 
gridCoverage2D.getRenderedImage().getData().getPixel(5, 5, (double[])null)[0], 
0.1);
+        assertEquals(4, gridCoverage2D.getNumSampleDimensions());
+    }
+}
\ No newline at end of file
diff --git 
a/common/src/test/java/org/apache/sedona/common/raster/FunctionsTest.java 
b/common/src/test/java/org/apache/sedona/common/raster/FunctionsTest.java
new file mode 100644
index 00000000..387d1448
--- /dev/null
+++ b/common/src/test/java/org/apache/sedona/common/raster/FunctionsTest.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.junit.Test;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Point;
+import org.opengis.referencing.operation.TransformException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+import static org.junit.Assert.*;
+
+public class FunctionsTest extends RasterTestBase {
+
+    @Test
+    public void envelope() {
+        Geometry envelope = Functions.envelope(oneBandRaster);
+        assertEquals(3600.0d, envelope.getArea(), 0.1d);
+        assertEquals(378922.0d + 30.0d, envelope.getCentroid().getX(), 0.1d);
+        assertEquals(4072345.0d + 30.0d, envelope.getCentroid().getY(), 0.1d);
+    }
+
+    @Test
+    public void testNumBands() {
+        assertEquals(1, Functions.numBands(oneBandRaster));
+        assertEquals(4, Functions.numBands(multiBandRaster));
+    }
+
+    @Test
+    public void value() throws TransformException {
+        assertNull("Points outside of the envelope should return null.", 
Functions.value(oneBandRaster, point(1, 1), 1));
+        assertNull("Invalid band should return null.", 
Functions.value(oneBandRaster, point(378923, 4072346), 0));
+        assertNull("Invalid band should return null.", 
Functions.value(oneBandRaster, point(378923, 4072346), 2));
+
+        Double value = Functions.value(oneBandRaster, point(378923, 4072346), 
1);
+        assertNotNull(value);
+        assertEquals(2.0d, value, 0.1d);
+
+        assertNull("Null should be returned for no data values.", 
Functions.value(oneBandRaster, point(378923, 4072376), 1));
+    }
+
+    @Test
+    public void valueWithMultibandRaster() throws TransformException {
+        // Multiband raster
+        assertEquals(9d, Functions.value(multiBandRaster, point(4.5d,4.5d), 
3), 0.1d);
+        assertEquals(255d, Functions.value(multiBandRaster, point(4.5d,4.5d), 
4), 0.1d);
+    }
+
+    @Test
+    public void values() throws TransformException {
+        // The function 'value' is implemented using 'values'.
+        // These test only cover bits not already covered by tests for 'value'
+        List<Geometry> points = Arrays.asList(new Geometry[]{point(378923, 
4072346), point(378924, 4072346)});
+        List<Double> values = Functions.values(oneBandRaster, points, 1);
+        assertEquals(2, values.size());
+        assertTrue(values.stream().allMatch(Objects::nonNull));
+
+        values = Functions.values(oneBandRaster, points, 0);
+        assertEquals(2, values.size());
+        assertTrue("All values should be null for invalid band index.", 
values.stream().allMatch(Objects::isNull));
+
+        values = Functions.values(oneBandRaster, points, 2);
+        assertEquals(2, values.size());
+        assertTrue("All values should be null for invalid band index.", 
values.stream().allMatch(Objects::isNull));
+
+        values = Functions.values(oneBandRaster, Arrays.asList(new 
Geometry[]{point(378923, 4072346), null}), 1);
+        assertEquals(2, values.size());
+        assertNull("Null geometries should return null values.", 
values.get(1));
+    }
+
+    private Point point(double x, double y) {
+        return new GeometryFactory().createPoint(new Coordinate(x, y));
+    }
+}
\ No newline at end of file
diff --git 
a/common/src/test/java/org/apache/sedona/common/raster/RasterTestBase.java 
b/common/src/test/java/org/apache/sedona/common/raster/RasterTestBase.java
new file mode 100644
index 00000000..ba19e1f1
--- /dev/null
+++ b/common/src/test/java/org/apache/sedona/common/raster/RasterTestBase.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.geotools.coverage.grid.GridCoverage2D;
+import org.geotools.coverage.grid.GridCoverageFactory;
+import org.geotools.gce.geotiff.GeoTiffWriter;
+import org.geotools.geometry.Envelope2D;
+import org.geotools.referencing.crs.DefaultGeographicCRS;
+import org.junit.Before;
+import org.opengis.parameter.GeneralParameterValue;
+
+import java.awt.*;
+import java.awt.image.BufferedImage;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+public class RasterTestBase {
+    String arc = "NCOLS 2\nNROWS 2\nXLLCORNER 378922\nYLLCORNER 
4072345\nCELLSIZE 30\nNODATA_VALUE 0\n0 1 2 3\n";
+    GridCoverage2D oneBandRaster;
+    GridCoverage2D multiBandRaster;
+    byte[] geoTiff;
+
+    @Before
+    public void setup() throws IOException {
+        oneBandRaster = 
Constructors.fromArcInfoAsciiGrid(arc.getBytes(StandardCharsets.UTF_8));
+        multiBandRaster = createMultibandRaster();
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        new GeoTiffWriter(bos).write(multiBandRaster, new 
GeneralParameterValue[]{});
+        geoTiff = bos.toByteArray();
+    }
+
+    GridCoverage2D createMultibandRaster() throws IOException {
+        GridCoverageFactory factory = new GridCoverageFactory();
+        BufferedImage image = new BufferedImage(10, 10, 
BufferedImage.TYPE_INT_ARGB);
+        for (int i = 0; i < image.getHeight(); i++) {
+            for (int j = 0; j < image.getWidth(); j++) {
+                int color = i + j;
+                image.setRGB(i, j, new Color(color, color, color).getRGB());
+            }
+        }
+        return factory.create("test", image, new 
Envelope2D(DefaultGeographicCRS.WGS84, 0, 0, 10, 10));
+    }
+}
diff --git 
a/common/src/test/java/org/apache/sedona/common/raster/SerdeTest.java 
b/common/src/test/java/org/apache/sedona/common/raster/SerdeTest.java
new file mode 100644
index 00000000..4500e11e
--- /dev/null
+++ b/common/src/test/java/org/apache/sedona/common/raster/SerdeTest.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sedona.common.raster;
+
+import org.geotools.coverage.grid.GridCoverage2D;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.*;
+
+public class SerdeTest extends RasterTestBase {
+
+    @Test
+    public void testRoundtripSerdeSingelbandRaster() throws IOException, 
ClassNotFoundException {
+        byte[] bytes = Serde.serialize(oneBandRaster);
+        GridCoverage2D raster = Serde.deserialize(bytes);
+        assertNotNull(raster);
+        assertEquals(1, raster.getNumSampleDimensions());
+    }
+
+    @Test
+    public void testRoundtripSerdeMultibandRaster() throws IOException, 
ClassNotFoundException {
+        byte[] bytes = Serde.serialize(this.multiBandRaster);
+        GridCoverage2D raster = Serde.deserialize(bytes);
+        assertNotNull(raster);
+        assertEquals(4, raster.getNumSampleDimensions());
+    }
+}
\ No newline at end of file
diff --git a/core/src/test/resources/raster_asc/test1.asc 
b/core/src/test/resources/raster_asc/test1.asc
new file mode 100644
index 00000000..8d164919
--- /dev/null
+++ b/core/src/test/resources/raster_asc/test1.asc
@@ -0,0 +1,7 @@
+NCOLS 2
+NROWS 2
+XLLCORNER 378922
+YLLCORNER 4072345
+CELLSIZE 30
+NODATA_VALUE 0
+0 1 2 3
diff --git a/docs/api/sql/Raster-loader.md b/docs/api/sql/Raster-loader.md
index d4c68447..aa72d80b 100644
--- a/docs/api/sql/Raster-loader.md
+++ b/docs/api/sql/Raster-loader.md
@@ -199,6 +199,39 @@ Output:
 !!!note
     Although the 3 RGB bands are mandatory, you can use [RS_Array(h*w, 
0.0)](#rs_array) to create an array (zeroed out, size = h * w) as input.
 
+
+## RS_FromArcInfoAsciiGrid
+
+Introduction: Returns a raster geometry from an Arc Info Ascii Grid file.
+
+Format: `RS_FromArcInfoAsciiGrid(asc: Array[Byte])`
+
+Since: `v1.4.0`
+
+Spark SQL example:
+
+```scala
+val df = spark.read.format("binaryFile").load("/some/path/*.asc")
+  .withColumn("raster", f.expr("RS_FromArcInfoAsciiGrid(content)"))
+```
+
+
+## RS_FromGeoTiff
+
+Introduction: Returns a raster geometry from a GeoTiff file.
+
+Format: `RS_FromGeoTiff(asc: Array[Byte])`
+
+Since: `v1.4.0`
+
+Spark SQL example:
+
+```scala
+val df = spark.read.format("binaryFile").load("/some/path/*.tiff")
+  .withColumn("raster", f.expr("RS_FromGeoTiff(content)"))
+```
+
+
 ## RS_GetBand
 
 Introduction: Return a particular band from Geotiff Dataframe
diff --git a/docs/api/sql/Raster-operators.md b/docs/api/sql/Raster-operators.md
index cda16c00..06db6b65 100644
--- a/docs/api/sql/Raster-operators.md
+++ b/docs/api/sql/Raster-operators.md
@@ -88,6 +88,23 @@ val multiplyDF = spark.sql("select RS_Divide(band1, band2) 
as divideBands from d
 
 ```
 
+## RS_Envelope
+
+Introduction: Returns the envelope of the raster as a Geometry.
+
+Format: `RS_Envelope (raster: Raster)`
+
+Since: `v1.4.0`
+
+Spark SQL example:
+```sql
+SELECT RS_Envelope(raster) FROM raster_table
+```
+Output:
+```
+POLYGON((0 0,20 0,20 60,0 60,0 0))
+```
+
 ## RS_FetchRegion
 
 Introduction: Fetch a subset of region from given Geotiff image based on 
minimumX, minimumY, maximumX and maximumY index as well original height and 
width of image
@@ -293,6 +310,24 @@ val normalizedDF = spark.sql("select 
RS_NormalizedDifference(band1, band2) as no
 
 ```
 
+## RS_NumBands
+
+Introduction: Returns the number of the bands in the raster.
+
+Format: `RS_NumBands (raster: Raster)`
+
+Since: `v1.4.0`
+
+Spark SQL example:
+```sql
+SELECT RS_NumBands(raster) FROM raster_table
+```
+
+Output:
+```
+4
+```
+
 ## RS_SquareRoot
 
 Introduction: Find Square root of band values in a geotiff image 
@@ -322,3 +357,78 @@ Spark SQL example:
 val subtractDF = spark.sql("select RS_Subtract(band1, band2) as 
differenceOfOfBands from dataframe")
 
 ```
+
+## RS_Value
+
+Introduction: Returns the value at the given point in the raster.
+If no band number is specified it defaults to 1. 
+
+Format: `RS_Value (raster: Raster, point: Geometry)`
+
+Format: `RS_Value (raster: Raster, point: Geometry, band: Int)`
+
+Since: `v1.4.0`
+
+Spark SQL example:
+```sql
+SELECT RS_Value(raster, ST_Point(-13077301.685, 4002565.802)) FROM raster_table
+```
+
+Output:
+```
+5.0
+```
+
+## RS_Values
+
+Introduction: Returns the values at the given points in the raster.
+If no band number is specified it defaults to 1.
+
+RS_Values is similar to RS_Value but operates on an array of points.
+RS_Values can be significantly faster since a raster only has to be loaded 
once for several points.
+
+Format: `RS_Values (raster: Raster, points: Array[Geometry])`
+
+Format: `RS_Values (raster: Raster, points: Array[Geometry], band: Int)`
+
+Since: `v1.4.0`
+
+Spark SQL example:
+```sql
+SELECT RS_Values(raster, Array(ST_Point(-1307.5, 400.8), ST_Point(-1403.3, 
399.1)))
+FROM raster_table
+```
+
+Output:
+```
+Array(5.0, 3.0)
+```
+
+Spark SQL example for joining a point dataset with a raster dataset:
+```scala
+val pointDf = spark.read...
+val rasterDf = spark.read.format("binaryFile").load("/some/path/*.tiff")
+  .withColumn("raster", expr("RS_FromGeoTiff(content)"))
+  .withColumn("envelope", expr("RS_Envelope(raster)"))
+
+// Join the points with the raster extent and aggregate points to arrays.
+// We only use the path and envelope of the raster to keep the shuffle as 
small as possible.
+val df = pointDf.join(rasterDf.select("path", "envelope"), 
expr("ST_Within(point_geom, envelope)"))
+  .groupBy("path")
+  .agg(collect_list("point_geom").alias("point"), 
collect_list("point_id").alias("id"))
+
+df.join(rasterDf, "path")
+  .selectExpr("explode(arrays_zip(id, point, RS_Values(raster, point))) as 
result")
+  .selectExpr("result.*")
+  .show()
+```
+
+Output:
+```
++----+------------+-------+
+| id | point      | value |
++----+------------+-------+
+|  4 | POINT(1 1) |   3.0 |
+|  5 | POINT(2 2) |   7.0 |
++----+------------+-------+
+```
diff --git a/pom.xml b/pom.xml
index 7bc4bc7d..82e3678c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -213,6 +213,12 @@
                 <version>${geotools.version}</version>
                 <scope>${geotools.scope}</scope>
             </dependency>
+            <dependency>
+                <groupId>org.geotools</groupId>
+                <artifactId>gt-arcgrid</artifactId>
+                <version>${geotools.version}</version>
+                <scope>${geotools.scope}</scope>
+            </dependency>
             <dependency>
                 <groupId>org.geotools</groupId>
                 <artifactId>gt-coverage</artifactId>
diff --git a/spark-shaded/pom.xml b/spark-shaded/pom.xml
index 3a5f9de9..59526f88 100644
--- a/spark-shaded/pom.xml
+++ b/spark-shaded/pom.xml
@@ -86,6 +86,10 @@
             <groupId>org.geotools</groupId>
             <artifactId>gt-geotiff</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.geotools</groupId>
+            <artifactId>gt-arcgrid</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.geotools</groupId>
             <artifactId>gt-coverage</artifactId>
diff --git a/sql/pom.xml b/sql/pom.xml
index 12461afd..0d4de07d 100644
--- a/sql/pom.xml
+++ b/sql/pom.xml
@@ -97,6 +97,10 @@
             <groupId>org.geotools</groupId>
             <artifactId>gt-coverage</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.geotools</groupId>
+            <artifactId>gt-arcgrid</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.locationtech.jts</groupId>
             <artifactId>jts-core</artifactId>
diff --git a/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala 
b/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
index f637cb34..3af49b67 100644
--- a/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
+++ b/sql/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
@@ -168,7 +168,13 @@ object Catalog {
     function[RS_HTML](),
     function[RS_Array](),
     function[RS_Normalize](),
-    function[RS_Append]()
+    function[RS_Append](),
+    function[RS_FromArcInfoAsciiGrid](),
+    function[RS_FromGeoTiff](),
+    function[RS_Envelope](),
+    function[RS_NumBands](),
+    function[RS_Value](1),
+    function[RS_Values](1)
   )
 
   val aggregateExpressions: Seq[Aggregator[Geometry, Geometry, Geometry]] = 
Seq(
diff --git 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/RasterUDT.scala 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/RasterUDT.scala
new file mode 100644
index 00000000..3411a49c
--- /dev/null
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/RasterUDT.scala
@@ -0,0 +1,22 @@
+package org.apache.spark.sql.sedona_sql.UDT
+
+
+import org.apache.sedona.common.raster.Serde
+import org.apache.spark.sql.types.{BinaryType, DataType, UserDefinedType}
+import org.geotools.coverage.grid.GridCoverage2D
+
+class RasterUDT extends UserDefinedType[GridCoverage2D] {
+  override def sqlType: DataType = BinaryType
+
+  override def serialize(raster: GridCoverage2D): Array[Byte] = 
Serde.serialize(raster)
+
+  override def deserialize(datum: Any): GridCoverage2D = {
+    datum match {
+      case bytes: Array[Byte] => Serde.deserialize(bytes)
+    }
+  }
+
+  override def userClass: Class[GridCoverage2D] = classOf[GridCoverage2D]
+}
+
+case object RasterUDT extends RasterUDT with Serializable
\ No newline at end of file
diff --git 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
index 1a8ed622..127205fa 100644
--- 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
+++ 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
@@ -18,13 +18,26 @@
  */
 package org.apache.spark.sql.sedona_sql.UDT
 
+import org.slf4j.{Logger, LoggerFactory}
 import org.apache.spark.sql.types.UDTRegistration
 import org.locationtech.jts.geom.Geometry
 import org.locationtech.jts.index.SpatialIndex
 
 object UdtRegistratorWrapper {
+
+  val logger: Logger = LoggerFactory.getLogger(getClass)
+
   def registerAll(): Unit = {
     UDTRegistration.register(classOf[Geometry].getName, 
classOf[GeometryUDT].getName)
     UDTRegistration.register(classOf[SpatialIndex].getName, 
classOf[IndexUDT].getName)
+    // Rasters requires geotools which is optional.
+    val gridClassName = "org.geotools.coverage.grid.GridCoverage2D"
+    try {
+      // Trigger an exception if geotools is not found.
+      java.lang.Class.forName(gridClassName, true, 
Thread.currentThread().getContextClassLoader)
+      UDTRegistration.register(gridClassName, classOf[RasterUDT].getName)
+    } catch {
+      case e: ClassNotFoundException => logger.warn("Geotools was not found on 
the classpath. Raster type will not be registered.")
+    }
   }
 }
diff --git 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Constructors.scala
 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Constructors.scala
new file mode 100644
index 00000000..37fb93ab
--- /dev/null
+++ 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Constructors.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.spark.sql.sedona_sql.expressions.raster
+
+import org.apache.sedona.common.raster.Constructors
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, 
Expression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.sedona_sql.UDT.RasterUDT
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType}
+import org.apache.spark.sql.sedona_sql.expressions.raster.implicits._
+
+
+case class RS_FromArcInfoAsciiGrid(inputExpressions: Seq[Expression]) extends 
Expression with CodegenFallback with ExpectsInputTypes {
+
+  override def nullable: Boolean = true
+
+  override def eval(input: InternalRow): Any = {
+    val bytes = inputExpressions(0).eval(input).asInstanceOf[Array[Byte]]
+    if (bytes == null) {
+      null
+    } else {
+      Constructors.fromArcInfoAsciiGrid(bytes).serialize
+    }
+  }
+
+  override def dataType: DataType = RasterUDT
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
+}
+
+case class RS_FromGeoTiff(inputExpressions: Seq[Expression]) extends 
Expression with CodegenFallback with ExpectsInputTypes {
+
+  override def nullable: Boolean = true
+
+  override def eval(input: InternalRow): Any = {
+    val bytes = inputExpressions(0).eval(input).asInstanceOf[Array[Byte]]
+    if (bytes == null) {
+      null
+    } else {
+      Constructors.fromGeoTiff(bytes).serialize
+    }
+  }
+
+  override def dataType: DataType = RasterUDT
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
+}
diff --git 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Functions.scala
 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Functions.scala
index 0c9a80f7..1e924304 100644
--- 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Functions.scala
+++ 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/Functions.scala
@@ -19,11 +19,16 @@
 
 package org.apache.spark.sql.sedona_sql.expressions.raster
 
+import org.apache.sedona.common.geometrySerde.GeometrySerializer
+import org.apache.sedona.common.raster.Functions
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
-import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeArrayData}
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, 
Expression}
 import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData}
+import org.apache.spark.sql.sedona_sql.UDT.{GeometryUDT, RasterUDT}
 import org.apache.spark.sql.sedona_sql.expressions.UserDataGeneratator
+import org.apache.spark.sql.sedona_sql.expressions.implicits._
+import org.apache.spark.sql.sedona_sql.expressions.raster.implicits._
 import org.apache.spark.sql.types._
 
 
@@ -804,4 +809,104 @@ case class RS_Append(inputExpressions: Seq[Expression])
   }
 }
 
+case class RS_Envelope(inputExpressions: Seq[Expression]) extends Expression 
with CodegenFallback with ExpectsInputTypes {
+  override def nullable: Boolean = true
 
+  override def eval(input: InternalRow): Any = {
+    val raster = inputExpressions(0).toRaster(input)
+    if (raster == null) {
+      null
+    } else {
+      Functions.envelope(raster).toGenericArrayData
+    }
+  }
+
+  override def dataType: DataType = GeometryUDT
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(RasterUDT)
+}
+
+case class RS_NumBands(inputExpressions: Seq[Expression]) extends Expression 
with CodegenFallback with ExpectsInputTypes {
+  override def nullable: Boolean = true
+
+  override def eval(input: InternalRow): Any = {
+    val raster = inputExpressions(0).toRaster(input)
+    if (raster == null) {
+      null
+    } else {
+      Functions.numBands(raster)
+    }
+  }
+
+  override def dataType: DataType = IntegerType
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(RasterUDT)
+}
+
+case class RS_Value(inputExpressions: Seq[Expression]) extends Expression with 
CodegenFallback with ExpectsInputTypes {
+
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = DoubleType
+
+  override def eval(input: InternalRow): Any = {
+    val raster = inputExpressions.head.toRaster(input)
+    val geom = inputExpressions(1).toGeometry(input)
+    val band = inputExpressions(2).eval(input).asInstanceOf[Int]
+    if (raster == null || geom == null) {
+      null
+    } else {
+      Functions.value(raster, geom, band)
+    }
+  }
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(RasterUDT, GeometryUDT, 
IntegerType)
+}
+
+case class RS_Values(inputExpressions: Seq[Expression]) extends Expression 
with CodegenFallback with ExpectsInputTypes {
+
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = ArrayType(DoubleType)
+
+  override def eval(input: InternalRow): Any = {
+    val raster = inputExpressions(0).toRaster(input)
+    val serializedGeometries = 
inputExpressions(1).eval(input).asInstanceOf[ArrayData]
+    val band = inputExpressions(2).eval(input).asInstanceOf[Int]
+    if (raster == null || serializedGeometries == null) {
+      null
+    } else {
+      val geometries = serializedGeometries.array.map {
+        case b: Array[Byte] => GeometrySerializer.deserialize(b)
+        case _ => null
+      }
+      new GenericArrayData(Functions.values(raster, 
java.util.Arrays.asList(geometries:_*), band).toArray)
+    }
+  }
+
+  override def children: Seq[Expression] = inputExpressions
+
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = 
{
+    copy(inputExpressions = newChildren)
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(RasterUDT, 
ArrayType(GeometryUDT), IntegerType)
+}
diff --git 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/implicits.scala
similarity index 53%
copy from 
sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
copy to 
sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/implicits.scala
index 1a8ed622..c4f4931a 100644
--- 
a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/UDT/UdtRegistratorWrapper.scala
+++ 
b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/implicits.scala
@@ -16,15 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.spark.sql.sedona_sql.UDT
+package org.apache.spark.sql.sedona_sql.expressions.raster
 
-import org.apache.spark.sql.types.UDTRegistration
-import org.locationtech.jts.geom.Geometry
-import org.locationtech.jts.index.SpatialIndex
+import org.apache.sedona.common.raster.Serde
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.geotools.coverage.grid.GridCoverage2D
 
-object UdtRegistratorWrapper {
-  def registerAll(): Unit = {
-    UDTRegistration.register(classOf[Geometry].getName, 
classOf[GeometryUDT].getName)
-    UDTRegistration.register(classOf[SpatialIndex].getName, 
classOf[IndexUDT].getName)
+object implicits {
+
+  implicit class RasterInputExpressionEnhancer(inputExpression: Expression) {
+    def toRaster(input: InternalRow): GridCoverage2D = {
+      inputExpression.eval(input).asInstanceOf[Array[Byte]] match {
+        case binary: Array[Byte] => Serde.deserialize(binary)
+        case _ => null
+      }
+    }
+  }
+
+  implicit class RasterEnhancer(raster: GridCoverage2D) {
+    def serialize: Array[Byte] = Serde.serialize(raster)
   }
 }
diff --git a/sql/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala 
b/sql/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala
index 582ae4ad..024e1f6c 100644
--- a/sql/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala
+++ b/sql/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala
@@ -18,6 +18,8 @@
  */
 package org.apache.sedona.sql
 
+import org.geotools.coverage.grid.GridCoverage2D
+import org.locationtech.jts.geom.Geometry
 import org.scalatest.{BeforeAndAfter, GivenWhenThen}
 
 import scala.collection.mutable
@@ -213,7 +215,7 @@ class rasteralgebraTest extends TestBaseScala with 
BeforeAndAfter with GivenWhen
       val rowFirst = df.first()
       val nBands = rowFirst.getAs[Int](1)
       val lengthInitial = 
rowFirst.getAs[mutable.WrappedArray[Double]](0).length
-      val lengthBand = lengthInitial/nBands
+      val lengthBand = lengthInitial / nBands
 
       df = df.selectExpr("data", "nBands", "RS_GetBand(data, 1, nBands) as 
band1", "RS_GetBand(data, 2, nBands) as band2")
       df = df.selectExpr("data", "nBands", "RS_NormalizedDifference(band2, 
band1) as normalizedDifference")
@@ -227,7 +229,7 @@ class rasteralgebraTest extends TestBaseScala with 
BeforeAndAfter with GivenWhen
       var rowFirst = df.first()
       val nBands = rowFirst.getAs[Int](1)
       val lengthInitial = 
rowFirst.getAs[mutable.WrappedArray[Double]](0).length
-      val lengthBand = lengthInitial/nBands
+      val lengthBand = lengthInitial / nBands
 
       df = df.selectExpr("data", "nBands", "RS_GetBand(data, 1, nBands) as 
band1", "RS_GetBand(data, 2, nBands) as band2")
       df = df.selectExpr("data", "nBands", "RS_NormalizedDifference(band2, 
band1) as normalizedDifference")
@@ -239,4 +241,76 @@ class rasteralgebraTest extends TestBaseScala with 
BeforeAndAfter with GivenWhen
     }
   }
 
+  describe("Should pass all raster function tests") {
+
+    it("Passed RS_FromGeoTiff should handle null values") {
+      val result = sparkSession.sql("select 
RS_FromGeoTiff(null)").first().get(0)
+      assert(result == null)
+    }
+
+    it("Passed RS_FromGeoTiff from binary") {
+      val df = sparkSession.read.format("binaryFile").load(resourceFolder + 
"raster/test1.tiff")
+      val result = df.selectExpr("RS_FromGeoTiff(content)").first().get(0)
+      assert(result != null)
+      assert(result.isInstanceOf[GridCoverage2D])
+    }
+
+    it("Passed RS_FromArcInfoAsciiGrid should handle null values") {
+      val result = sparkSession.sql("select 
RS_FromArcInfoAsciiGrid(null)").first().get(0)
+      assert(result == null)
+    }
+
+    it("Passed RS_FromArcInfoAsciiGrid from binary") {
+      val df = sparkSession.read.format("binaryFile").load(resourceFolder + 
"raster_asc/test1.asc")
+      val result = 
df.selectExpr("RS_FromArcInfoAsciiGrid(content)").first().get(0)
+      assert(result != null)
+    }
+
+    it("Passed RS_Envelope should handle null values") {
+      val result = sparkSession.sql("select RS_Envelope(null)").first().get(0)
+      assert(result == null)
+    }
+
+    it("Passed RS_Envelope with raster") {
+      val df = sparkSession.read.format("binaryFile").load(resourceFolder + 
"raster/test1.tiff")
+      val result = 
df.selectExpr("RS_Envelope(RS_FromGeoTiff(content))").first().get(0)
+      assert(result != null)
+      assert(result.isInstanceOf[Geometry])
+    }
+
+    it("Passed RS_NumBands should handle null values") {
+      val result = sparkSession.sql("select RS_NumBands(null)").first().get(0)
+      assert(result == null)
+    }
+
+    it("Passed RS_NumBands with raster") {
+      val df = sparkSession.read.format("binaryFile").load(resourceFolder + 
"raster/test1.tiff")
+      val result = 
df.selectExpr("RS_NumBands(RS_FromGeoTiff(content))").first().getInt(0)
+      assert(result == 1)
+    }
+
+    it("Passed RS_Value should handle null values") {
+      val result = sparkSession.sql("select RS_Value(null, 
null)").first().get(0)
+      assert(result == null)
+    }
+
+    it("Passed RS_Value with raster") {
+      val df = sparkSession.read.format("binaryFile").load(resourceFolder + 
"raster/test1.tiff")
+      val result = df.selectExpr("RS_Value(RS_FromGeoTiff(content), 
ST_Point(-13077301.685, 4002565.802))").first().getDouble(0)
+      assert(result == 255d)
+    }
+
+    it("Passed RS_Values should handle null values") {
+      val result = sparkSession.sql("select RS_Values(null, 
null)").first().get(0)
+      assert(result == null)
+    }
+
+    it("Passed RS_Values with raster") {
+      val df = sparkSession.read.format("binaryFile").load(resourceFolder + 
"raster/test1.tiff")
+      val result = df.selectExpr("RS_Values(RS_FromGeoTiff(content), 
array(ST_Point(-13077301.685, 4002565.802), null))").first().getList[Any](0)
+      assert(result.size() == 2)
+      assert(result.get(0) == 255d)
+      assert(result.get(1) == null)
+    }
+  }
 }


Reply via email to