jiayuasu commented on issue #814:
URL: https://github.com/apache/sedona/issues/814#issuecomment-1495452309

   @conect 
   
   Test data:
   
   ```
   { "type": "FeatureCollection",
       "features": [
         { "type": "Feature",
           "geometry": {"type": "Point", "coordinates": [102.0, 0.5]},
           "properties": {"prop0": "value0"}
           },
         { "type": "Feature",
           "geometry": {
             "type": "LineString",
             "coordinates": [
               [102.0, 0.0], [103.0, 1.0], [104.0, 0.0], [105.0, 1.0]
               ]
             },
           "properties": {
             "prop0": "value1",
             "prop1": 0.0
             }
           },
         { "type": "Feature",
            "geometry": {
              "type": "Polygon",
              "coordinates": [
                [ [100.0, 0.0], [101.0, 0.0], [101.0, 1.0],
                  [100.0, 1.0], [100.0, 0.0] ]
                ]
            },
            "properties": {
              "prop0": "value2",
              "prop1": {"this": "that"}
              }
            }
          ]
        }
   ```
   
   Code:
   
   ```scala
         val schema = "type string, features array<struct<type string, geometry 
string, properties map<string, string>>>"
         val df = sparkSession.read.schema(schema).option("multiLine", 
"true").json(geojsonFeatureCollection)
         val parsedDf = df.selectExpr("explode(features) as 
features").select("features.*")
           .withColumn("geometry", expr("ST_GeomFromGeoJSON(geometry)"))
           .withColumn("prop0", 
expr("properties['prop0']")).drop("properties").drop("type")
         parsedDf.show()
         parsedDf.printSchema()
   ```
   
   Output
   
   ```
   +--------------------+------+
   |            geometry| prop0|
   +--------------------+------+
   |     POINT (102 0.5)|value0|
   |LINESTRING (102 0...|value1|
   |POLYGON ((100 0, ...|value2|
   +--------------------+------+
   
   root
    |-- geometry: geometry (nullable = false)
    |-- prop0: string (nullable = true)
   
   
   ```
   
   To SpatialRDD and saveAsGeoJSON again
   
   ```
         var spatialRDD = Adapter.toSpatialRdd(parsedDf, "geometry")
         spatialRDD.saveAsGeoJSON("testOutput.json")
   ```
   
   testOutput.json content
   ```
   
{"type":"Feature","geometry":{"type":"Point","coordinates":[102.0,0.5]},"properties":{"prop0":"value0"}}
   
{"type":"Feature","geometry":{"type":"LineString","coordinates":[[102.0,0.0],[103.0,1.0],[104.0,0.0],[105.0,1.0]]},"properties":{"prop0":"value1"}}
   
{"type":"Feature","geometry":{"type":"Polygon","coordinates":[[[100.0,0.0],[101.0,0.0],[101.0,1.0],[100.0,1.0],[100.0,0.0]]]},"properties":{"prop0":"value2"}}
   ```
   
   This works for Scala, Java, Python, R. The detailed explanation can be found 
here: 
https://sedona.apache.org/1.4.0/tutorial/sql/#load-geojson-using-spark-json-data-source


-- 
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