jiayuasu commented on code in PR #612:
URL: https://github.com/apache/incubator-sedona/pull/612#discussion_r854661548


##########
docs/api/sql/Raster-loader.md:
##########
@@ -142,3 +170,82 @@ Output:
 +--------------------+
 ```
 
+### Geotiff Dataframe Writer
+
+Introduction: You can write a GeoTiff dataframe as GeoTiff images using the 
spark `write` feature with the format `geotiff`.
+
+Since: `v1.2.1`
+
+Spark SQL example:
+
+The schema of the GeoTiff dataframe to be written can be one of the following 
two schemas:
+
+```html
+ |-- image: struct (nullable = true)
+ |    |-- origin: string (nullable = true)
+ |    |-- Geometry: geometry (nullable = true)
+ |    |-- height: integer (nullable = true)
+ |    |-- width: integer (nullable = true)
+ |    |-- nBands: integer (nullable = true)
+ |    |-- data: array (nullable = true)
+ |    |    |-- element: double (containsNull = true)
+```
+
+or
+
+```html
+ |-- origin: string (nullable = true)
+ |-- Geometry: geometry (nullable = true)
+ |-- height: integer (nullable = true)
+ |-- width: integer (nullable = true)
+ |-- nBands: integer (nullable = true)
+ |-- data: array (nullable = true)
+ |    |-- element: double (containsNull = true)
+```
+
+Field names can be renamed, but schema should exactly match with one of the 
above two schemas. The output path could be a path to a directory where GeoTiff 
images will be saved. If the directory already exists, `write` should be called 
in `overwrite` mode.
+
+```Scala
+var dfToWrite = sparkSession.read.format("geotiff").option("dropInvalid", 
true).option("readToCRS", "EPSG:4326").load("PATH_TO_INPUT_GEOTIFF_IMAGES")
+dfToWrite.write.format("geotiff").save("DESTINATION_PATH")
+```
+
+You can override an existing path with the following approach:
+
+```Scala
+dfToWrite.write.mode("overwrite").format("geotiff").save("DESTINATION_PATH")
+```
+
+You can also extract the columns nested within `image` column and write the 
dataframe as GeoTiff image.
+
+```Scala
+dfToWrite = dfToWrite.selectExpr("image.origin as origin","image.wkt as wkt", 
"image.height as height", "image.width as width", "image.data as data", 
"image.nBands as nBands")
+dfToWrite.write.mode("overwrite").format("geotiff").save("DESTINATION_PATH")
+```
+
+If you want the saved GeoTiff images not to be distributed into multiple 
partitions, you can call coalesce to merge all files in a single partition.
+
+```Scala
+dfToWrite.coalesce(1).write.mode("overwrite").format("geotiff").save("DESTINATION_PATH")
+```
+
+In case, you rename the columns of GeoTiff dataframe, you can set the 
corresponding column names with the `option` parameter. All available optional 
parameters are listed below:
+
+```html
+ |-- writeToCRS: (Default value "EPSG:4326") => Coordinate reference system of 
the geometry coordinates representing the location of the Geotiff.
+ |-- fieldImage: (Default value "image") => Indicates the image column of 
GeoTiff DataFrame.
+ |-- fieldOrigin: (Default value "origin") => Indicates the origin column of 
GeoTiff DataFrame.
+ |-- fieldNBands: (Default value "nBands") => Indicates the nBands column of 
GeoTiff DataFrame.
+ |-- fieldWidth: (Default value "width") => Indicates the width column of 
GeoTiff DataFrame.
+ |-- fieldHeight: (Default value "height") => Indicates the height column of 
GeoTiff DataFrame.
+ |-- fieldWkt: (Default value "wkt") => Indicates the wkt column of GeoTiff 
DataFrame.

Review Comment:
   this should be fieldGeometry. Please fix it in the code as well.



##########
sql/src/main/scala/org/apache/spark/sql/sedona_sql/io/GeotiffFileFormat.scala:
##########
@@ -101,4 +129,127 @@ private[spark] class GeotiffFileFormat extends FileFormat 
with DataSourceRegiste
       }
     }
   }
+
+  private def isValidGeoTiffSchema(imageWriteOptions: ImageWriteOptions, 
dataSchema: StructType): Boolean = {
+    val fields = dataSchema.fieldNames
+    if (fields.contains(imageWriteOptions.colImage) ){
+      val schemaFields = 
dataSchema.fields(dataSchema.fieldIndex(imageWriteOptions.colImage)).dataType.asInstanceOf[StructType]
+      if (schemaFields.fieldNames.length != 6) return false
+    }
+    else {
+      if (fields.length != 6) return false
+    }
+    true
+  }
+
+}
+
+// class for writing geoTiff images
+private class GeotiffFileWriter(savePath: String,
+                                imageWriteOptions: ImageWriteOptions,
+                                dataSchema: StructType,
+                                context: TaskAttemptContext) extends 
OutputWriter {
+
+  // set writing parameters
+  private val DEFAULT_WRITE_PARAMS: GeoTiffWriteParams = new 
GeoTiffWriteParams()
+  DEFAULT_WRITE_PARAMS.setCompressionMode(ImageWriteParam.MODE_EXPLICIT)
+  DEFAULT_WRITE_PARAMS.setCompressionType("LZW")
+  DEFAULT_WRITE_PARAMS.setCompressionQuality(0.75F)
+  DEFAULT_WRITE_PARAMS.setTilingMode(ImageWriteParam.MODE_EXPLICIT)
+  DEFAULT_WRITE_PARAMS.setTiling(512, 512)
+
+  private val hfs = new Path(savePath).getFileSystem(context.getConfiguration)
+
+  override def write(row: InternalRow): Unit = {
+    // retrieving the metadata of a geotiff image
+    println("Schema:")

Review Comment:
   Don't use println to print log info. Use Log4j instead. See the example: 
https://github.com/apache/incubator-sedona/blob/master/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala#L1004
   
   Use "info" level in logging. To see the log in the test, set log level to 
info level in TestBase.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to