You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sedona.apache.org by ji...@apache.org on 2023/09/13 17:24:23 UTC

[sedona] branch master updated: [SEDONA-393] Add RS_AsPNG (#1016)

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 11286988d [SEDONA-393] Add RS_AsPNG (#1016)
11286988d is described below

commit 11286988d406d01c7ca8b19481d2eda81c53a98a
Author: Furqaanahmed Khan <46...@users.noreply.github.com>
AuthorDate: Wed Sep 13 13:24:18 2023 -0400

    [SEDONA-393] Add RS_AsPNG (#1016)
---
 .../apache/sedona/common/raster/RasterOutputs.java |  7 ++++++
 .../sedona/common/raster/RasterOutputTest.java     | 14 ++++++++++++
 docs/api/sql/Raster-writer.md                      | 26 ++++++++++++++++++++++
 .../scala/org/apache/sedona/sql/UDF/Catalog.scala  |  1 +
 .../expressions/raster/RasterOutputs.scala         |  6 +++++
 .../org/apache/sedona/sql/rasteralgebraTest.scala  | 14 ++++++++++++
 6 files changed, 68 insertions(+)

diff --git a/common/src/main/java/org/apache/sedona/common/raster/RasterOutputs.java b/common/src/main/java/org/apache/sedona/common/raster/RasterOutputs.java
index a1296f9cd..3f76f88d6 100644
--- a/common/src/main/java/org/apache/sedona/common/raster/RasterOutputs.java
+++ b/common/src/main/java/org/apache/sedona/common/raster/RasterOutputs.java
@@ -93,6 +93,13 @@ public class RasterOutputs
         return true;
     }
 
+    public static byte[] asPNG(GridCoverage2D raster) throws IOException {
+        RenderedImage renderedImage = raster.getRenderedImage();
+        ByteArrayOutputStream os = new ByteArrayOutputStream();
+        ImageIO.write(renderedImage, "png", os);
+        return os.toByteArray();
+    }
+
     public static byte[] asArcGrid(GridCoverage2D raster, int sourceBand) {
         ByteArrayOutputStream out = new ByteArrayOutputStream();
         GridCoverageWriter writer;
diff --git a/common/src/test/java/org/apache/sedona/common/raster/RasterOutputTest.java b/common/src/test/java/org/apache/sedona/common/raster/RasterOutputTest.java
index 88c6d7541..7375b9396 100644
--- a/common/src/test/java/org/apache/sedona/common/raster/RasterOutputTest.java
+++ b/common/src/test/java/org/apache/sedona/common/raster/RasterOutputTest.java
@@ -20,9 +20,11 @@ package org.apache.sedona.common.raster;
 
 import org.geotools.coverage.grid.GridCoverage2D;
 import org.junit.Test;
+import org.opengis.referencing.FactoryException;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URLConnection;
 
 import static org.junit.Assert.*;
 
@@ -38,6 +40,18 @@ public class RasterOutputTest
         assertTrue(resultRaw.startsWith("iVBORw0KGgoAAAANSUhEUgAABaAAAALQCAMAAABR+ye1AAADAFBMVEXE9/W48vOq7PGa5u6L3"));
     }
 
+    @Test
+    public void testAsPNG() throws IOException, FactoryException {
+        String dirPath = System.getProperty("user.dir") + "/target/testAsPNGFunction/";
+        new File(dirPath).mkdirs();
+        GridCoverage2D raster = rasterFromGeoTiff(resourceFolder + "raster_geotiff_color/FAA_UTM18N_NAD83.tif");
+        byte[] pngData = RasterOutputs.asPNG(raster);
+        RasterOutputs.writeToDiskFile(pngData, dirPath + "test1.png");
+        File f = new File(dirPath + "test1.png");
+        String mimeType = URLConnection.guessContentTypeFromName(f.getName());
+        assertEquals("image/png", mimeType);
+    }
+
     @Test
     public void testAsGeoTiff() throws IOException {
         GridCoverage2D rasterOg = rasterFromGeoTiff(resourceFolder + "raster/test1.tiff");
diff --git a/docs/api/sql/Raster-writer.md b/docs/api/sql/Raster-writer.md
index 348174bb3..813fd8d1f 100644
--- a/docs/api/sql/Raster-writer.md
+++ b/docs/api/sql/Raster-writer.md
@@ -93,6 +93,32 @@ root
  |-- geotiff: binary (nullable = true)
 ```
 
+#### RS_AsPNG
+
+Introduction: Returns a PNG byte array, that can be written to raster files as PNGs using the [sedona function](#write-a-binary-dataframe-to-raster-files). This function can only accept pixel data type of unsigned integer.
+
+!!!Note
+	Raster having `UNSIGNED_8BITS` pixel data type will have range of `0 - 255`, whereas rasters having `UNSIGNED_16BITS` pixel data type will have range of `0 - 65535`. If provided pixel value is greater than either `255` for `UNSIGNED_8BITS` or `65535` for `UNSIGNED_16BITS`, then the extra bit will be truncated.
+
+!!!Note
+	Raster that have float or double values will result in an empty byte array. PNG only accepts Integer values, if you want to write your raster to an image file, please refer to [RS_AsGeoTiff](#rs_asgeotiff).
+
+Since: `v1.5.0`
+
+Format: `RS_AsPNG(raster: Raster)`
+
+Spark SQL Example:
+
+```sql
+SELECT RS_AsPNG(raster) FROM Rasters
+```
+
+Output:
+
+```
+[-119, 80, 78, 71, 13, 10, 26, 10, 0, 0, 0, 13, 73...]
+```
+
 ### Write a binary DataFrame to raster files
 
 Introduction: You can write a Sedona binary DataFrame to external storage using Sedona's built-in `raster` data source. Note that: `raster` data source does not support reading rasters. Please use Spark built-in `binaryFile` and Sedona RS constructors together to read rasters.
diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala b/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
index d90588931..b8be7a051 100644
--- a/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
+++ b/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala
@@ -213,6 +213,7 @@ object Catalog {
     function[RS_AsRaster](),
     function[RS_AsArcGrid](),
     function[RS_AsBase64](),
+    function[RS_AsPNG](),
     function[RS_Width](),
     function[RS_Height](),
     function[RS_UpperLeftX](),
diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterOutputs.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterOutputs.scala
index 638a31438..776c4bef0 100644
--- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterOutputs.scala
+++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/raster/RasterOutputs.scala
@@ -39,6 +39,12 @@ case class RS_AsArcGrid(inputExpressions: Seq[Expression])
   }
 }
 
+case class RS_AsPNG(inputExpressions: Seq[Expression]) extends InferredExpression(RasterOutputs.asPNG _) {
+  protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = {
+    copy(inputExpressions = newChildren)
+  }
+}
+
 case class RS_AsBase64(inputExpressions: Seq[Expression]) extends InferredExpression(RasterOutputs.asBase64 _) {
   protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = {
     copy(inputExpressions = newChildren)
diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala b/spark/common/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala
index 0ca905427..aa3c72217 100644
--- a/spark/common/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala
+++ b/spark/common/src/test/scala/org/apache/sedona/sql/rasteralgebraTest.scala
@@ -20,6 +20,7 @@ package org.apache.sedona.sql
 
 import org.apache.sedona.common.raster.MapAlgebra
 import org.apache.sedona.common.utils.RasterUtils
+import org.apache.spark.sql.SaveMode
 import org.apache.spark.sql.functions.{collect_list, expr}
 import org.geotools.coverage.grid.GridCoverage2D
 import org.junit.Assert.{assertEquals, assertNull}
@@ -27,6 +28,8 @@ import org.locationtech.jts.geom.{Coordinate, Geometry}
 import org.scalatest.{BeforeAndAfter, GivenWhenThen}
 
 import java.awt.image.DataBuffer
+import java.io.File
+import java.net.URLConnection
 import scala.collection.mutable
 
 
@@ -517,6 +520,17 @@ class rasteralgebraTest extends TestBaseScala with BeforeAndAfter with GivenWhen
       assertEquals(resultRaw.asInstanceOf[GridCoverage2D].getEnvelope.toString, resultLoaded.asInstanceOf[GridCoverage2D].getEnvelope.toString)
     }
 
+    it("Passed RS_AsPNG") {
+      val dirPath = System.getProperty("user.dir") + "/target/testAsPNGFunction/"
+      new File(dirPath).mkdirs()
+      var df = sparkSession.read.format("binaryFile").load(resourceFolder + "raster_geotiff_color/FAA_UTM18N_NAD83.tif")
+      df = df.selectExpr("RS_AsPNG(RS_FromGeoTiff(content)) as raster")
+      df.write.format("raster").option("rasterField", "raster").option("fileExtension", ".png").mode(SaveMode.Overwrite).save(dirPath)
+      val f = new File(dirPath + "part-*/*.png")
+      val mimeType = URLConnection.guessContentTypeFromName(f.getName)
+      assertEquals("image/png", mimeType)
+    }
+
     it("Passed RS_AsArcGrid with different bands") {
       val df = sparkSession.read.format("binaryFile").load(resourceFolder + "raster_geotiff_color/*").selectExpr("RS_FromGeoTiff(content) as raster")
       val rasterDf = df.selectExpr("RS_AsArcGrid(raster, 0) as arc", "RS_AsArcGrid(raster, 1) as arc2")