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/01/06 23:31:24 UTC

[sedona] branch master updated: [SEDONA-230] rdd.saveAsGeoJSON should generate feature properties with field names (#746)

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 4d23fb46 [SEDONA-230] rdd.saveAsGeoJSON should generate feature properties with field names (#746)
4d23fb46 is described below

commit 4d23fb4649b6f05fd13c9880bb04a2fbc4f1c4e9
Author: Jia Yu <ji...@apache.org>
AuthorDate: Fri Jan 6 16:31:20 2023 -0700

    [SEDONA-230] rdd.saveAsGeoJSON should generate feature properties with field names (#746)
---
 R/tests/testthat/helper-initialize.R               | 15 +++++
 R/tests/testthat/test-data-interface.R             | 19 +++++-
 .../org/apache/sedona/common/utils/GeomUtils.java  | 28 ++++++++
 .../sedona/core/formatMapper/FormatUtils.java      | 29 ++++----
 .../apache/sedona/core/spatialRDD/SpatialRDD.java  | 43 ++++++------
 .../test/java/org/apache/sedona/core/TestBase.java | 13 ++++
 .../{GeoJsonReaderTest.java => GeoJsonIOTest.java} | 78 ++++++++++++++++++++--
 .../TestReadInvalidSyntaxGeometriesTest.java       |  2 +-
 8 files changed, 183 insertions(+), 44 deletions(-)

diff --git a/R/tests/testthat/helper-initialize.R b/R/tests/testthat/helper-initialize.R
index 521d800a..d18d1b17 100644
--- a/R/tests/testthat/helper-initialize.R
+++ b/R/tests/testthat/helper-initialize.R
@@ -123,6 +123,21 @@ expect_geom_equal <- function(sc, lhs, rhs) {
   }
 }
 
+expect_geom_equal_geojson <- function(sc, lhs, rhs) {
+  testthat::expect_equal(length(lhs), length(rhs))
+  for (i in seq_along(lhs)) {
+    testthat::expect_true(
+      invoke_static(
+        sc,
+        "org.apache.sedona.common.utils.GeomUtils",
+        "equalsExactGeomUnsortedUserData",
+        lhs[[i]],
+        rhs[[i]]
+      )
+    )
+  }
+}
+
 as.coordinate_list <- function(geometry) {
   geometry %>%
     invoke("getCoordinates") %>%
diff --git a/R/tests/testthat/test-data-interface.R b/R/tests/testthat/test-data-interface.R
index 1f908d9c..d6a58bef 100644
--- a/R/tests/testthat/test-data-interface.R
+++ b/R/tests/testthat/test-data-interface.R
@@ -51,6 +51,21 @@ expect_result_matches_original <- function(pt_rdd) {
   )
 }
 
+expect_result_matches_original_geojson <- function(pt_rdd) {
+  expect_equal(
+    pt_rdd %>% invoke("%>%", list("rawSpatialRDD"), list("count")),
+    test_rdd_with_non_spatial_attrs$.jobj %>%
+      invoke("%>%", list("rawSpatialRDD"), list("count"))
+  )
+  expect_geom_equal_geojson(
+    sc,
+    test_rdd_with_non_spatial_attrs$.jobj %>%
+      invoke("%>%", list("rawSpatialRDD"), list("takeOrdered", 5L), list("toArray")),
+    pt_rdd %>%
+      invoke("%>%", list("rawSpatialRDD"), list("takeOrdered", 5L), list("toArray"))
+  )
+}
+
 test_that("sedona_read_dsv_to_typed_rdd() creates PointRDD correctly", {
   pt_rdd <- sedona_read_dsv_to_typed_rdd(
     sc,
@@ -319,11 +334,11 @@ test_that("sedona_write_geojson() works as expected", {
     sc$state$object_cache$storage_levels$memory_only
   )
 
-  expect_result_matches_original(pt_rdd)
+  expect_result_matches_original_geojson(pt_rdd)
 })
 
 test_that("sedona_save_spatial_rdd() works as expected", {
-  for (fmt in c("wkb", "wkt", "geojson")) {
+  for (fmt in c("wkb", "wkt")) {
     location <- tempfile(pattern = "pt_", fileext = paste0(".", fmt))
     copy_to(
       sc, tibble::tibble(id = 1, name = "a point", type = "point")
diff --git a/common/src/main/java/org/apache/sedona/common/utils/GeomUtils.java b/common/src/main/java/org/apache/sedona/common/utils/GeomUtils.java
index 1f60c533..535cef16 100644
--- a/common/src/main/java/org/apache/sedona/common/utils/GeomUtils.java
+++ b/common/src/main/java/org/apache/sedona/common/utils/GeomUtils.java
@@ -56,6 +56,34 @@ public class GeomUtils {
         else return false;
     }
 
+    /**
+     * This is for verifying the correctness of two geometries loaded from geojson
+     * @param geom1
+     * @param geom2
+     * @return
+     */
+    public static boolean equalsExactGeomUnsortedUserData(Geometry geom1, Object geom2) {
+        if (! (geom2 instanceof Geometry)) return false;
+        Geometry g = (Geometry) geom2;
+        if (equalsUserData(geom1.getUserData(), g.getUserData())) return geom1.equalsExact(g);
+        else return false;
+    }
+
+    /**
+     * Use for check if two user data attributes are equal
+     * This is mainly used for GeoJSON parser as the column order is uncertain each time
+     * @param userData1
+     * @param userData2
+     * @return
+     */
+    public static boolean equalsUserData(Object userData1, Object userData2) {
+        String[] split1 = ((String) userData1).split("\t");
+        String[] split2 = ((String) userData2).split("\t");
+        Arrays.sort(split1);
+        Arrays.sort(split2);
+        return Arrays.equals(split1, split2);
+    }
+
     /**
      * Swaps the XY coordinates of a geometry.
      */
diff --git a/core/src/main/java/org/apache/sedona/core/formatMapper/FormatUtils.java b/core/src/main/java/org/apache/sedona/core/formatMapper/FormatUtils.java
index 525f30b3..0b0aca57 100644
--- a/core/src/main/java/org/apache/sedona/core/formatMapper/FormatUtils.java
+++ b/core/src/main/java/org/apache/sedona/core/formatMapper/FormatUtils.java
@@ -119,23 +119,22 @@ public class FormatUtils<T extends Geometry> implements Serializable {
 
     public static List<String> readGeoJsonPropertyNames(String geoJson)
     {
+        if (geoJson == null) {
+            logger.warn("The given GeoJson record is null and cannot be used to find property names");
+            return null;
+        }
+        List<String> propertyList = new ArrayList<>();
         if (geoJson.contains("Feature") || geoJson.contains("feature") || geoJson.contains("FEATURE")) {
-            if (geoJson.contains("properties")) {
-                Feature feature = (Feature) GeoJSONFactory.create(geoJson);
-                if (Objects.isNull(feature.getId())) {
-                    return new ArrayList(feature.getProperties().keySet());
-                }
-                else {
-                    List<String> propertyList = new ArrayList<>(Arrays.asList("id"));
-                    for (String geoJsonProperty : feature.getProperties().keySet()) {
-                        propertyList.add(geoJsonProperty);
-                    }
-                    return propertyList;
-                }
+            Feature feature = (Feature) GeoJSONFactory.create(geoJson);
+            if (!Objects.isNull(feature.getId())) {
+                propertyList.add("id");
+            }
+            Map<String, Object> properties = feature.getProperties();
+            if (properties != null) {
+                propertyList.addAll(properties.keySet());
             }
         }
-        logger.warn("[Sedona] The GeoJSON file doesn't have feature properties");
-        return null;
+        return propertyList.size() > 0 ? propertyList:null;
     }
 
     private void readObject(ObjectInputStream inputStream)
@@ -173,7 +172,7 @@ public class FormatUtils<T extends Geometry> implements Serializable {
                 for (Object property : featurePropertiesproperties.values()
                 ) {
                     if (property == null) {
-                        nonSpatialData.add("null");
+                        nonSpatialData.add("");
                     }
                     else {
                         nonSpatialData.add(property.toString());
diff --git a/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java b/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
index 4f1123d5..edfa60e7 100644
--- a/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
+++ b/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
@@ -56,6 +56,7 @@ import scala.Tuple2;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -583,30 +584,34 @@ public class SpatialRDD<T extends Geometry>
      */
     public void saveAsGeoJSON(String outputLocation)
     {
-        this.rawSpatialRDD.mapPartitions(new FlatMapFunction<Iterator<T>, String>()
-        {
-            @Override
-            public Iterator<String> call(Iterator<T> iterator)
-                    throws Exception
-            {
-                ArrayList<String> result = new ArrayList();
-                GeoJSONWriter writer = new GeoJSONWriter();
-                while (iterator.hasNext()) {
-                    Geometry spatialObject = iterator.next();
-                    Feature jsonFeature;
-                    if (spatialObject.getUserData() != null) {
-                        Map<String, Object> userData = new HashMap<String, Object>();
-                        userData.put("UserData", spatialObject.getUserData());
-                        jsonFeature = new Feature(writer.write(spatialObject), userData);
+        this.rawSpatialRDD.mapPartitions((FlatMapFunction<Iterator<T>, String>) iterator -> {
+            ArrayList<String> result = new ArrayList();
+            GeoJSONWriter writer = new GeoJSONWriter();
+            while (iterator.hasNext()) {
+                Geometry spatialObject = iterator.next();
+                Feature jsonFeature;
+                if (spatialObject.getUserData() != null) {
+                    Map<String, Object> fields = new HashMap<String, Object>();
+                    String[] fieldValues = spatialObject.getUserData().toString().split("\t");
+                    if (fieldNames != null && fieldValues.length == fieldNames.size()) {
+                        for (int i = 0 ; i < fieldValues.length ; i++) {
+                            fields.put(fieldNames.get(i), fieldValues[i]);
+                        }
                     }
                     else {
-                        jsonFeature = new Feature(writer.write(spatialObject), null);
+                        for (int i = 0 ; i < fieldValues.length ; i++) {
+                            fields.put("_c" + i, fieldValues[i]);
+                        }
                     }
-                    String jsonstring = jsonFeature.toString();
-                    result.add(jsonstring);
+                    jsonFeature = new Feature(writer.write(spatialObject), fields);
+                }
+                else {
+                    jsonFeature = new Feature(writer.write(spatialObject), null);
                 }
-                return result.iterator();
+                String jsonstring = jsonFeature.toString();
+                result.add(jsonstring);
             }
+            return result.iterator();
         }).saveAsTextFile(outputLocation);
     }
 
diff --git a/core/src/test/java/org/apache/sedona/core/TestBase.java b/core/src/test/java/org/apache/sedona/core/TestBase.java
index 7403bad0..1ef1b71a 100644
--- a/core/src/test/java/org/apache/sedona/core/TestBase.java
+++ b/core/src/test/java/org/apache/sedona/core/TestBase.java
@@ -19,6 +19,9 @@
 
 package org.apache.sedona.core;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.sedona.core.serde.SedonaKryoRegistrator;
@@ -26,6 +29,8 @@ import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.serializer.KryoSerializer;
 
+import java.io.IOException;
+
 public class TestBase
 {
     protected static SparkConf conf;
@@ -41,5 +46,13 @@ public class TestBase
         Logger.getLogger("org").setLevel(Level.WARN);
         Logger.getLogger("akka").setLevel(Level.WARN);
     }
+
+    protected static void deleteFile(String path)
+            throws IOException
+    {
+        Configuration hadoopConfig = new Configuration();
+        FileSystem fileSystem = FileSystem.get(hadoopConfig);
+        fileSystem.delete(new Path(path), true);
+    }
 }
 
diff --git a/core/src/test/java/org/apache/sedona/core/formatMapper/GeoJsonReaderTest.java b/core/src/test/java/org/apache/sedona/core/formatMapper/GeoJsonIOTest.java
similarity index 52%
rename from core/src/test/java/org/apache/sedona/core/formatMapper/GeoJsonReaderTest.java
rename to core/src/test/java/org/apache/sedona/core/formatMapper/GeoJsonIOTest.java
index 423431ed..a07491a3 100644
--- a/core/src/test/java/org/apache/sedona/core/formatMapper/GeoJsonReaderTest.java
+++ b/core/src/test/java/org/apache/sedona/core/formatMapper/GeoJsonIOTest.java
@@ -24,12 +24,17 @@ import org.apache.sedona.core.spatialRDD.SpatialRDD;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.locationtech.jts.geom.Geometry;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 
-public class GeoJsonReaderTest
+public class GeoJsonIOTest
         extends TestBase
 {
 
@@ -43,12 +48,12 @@ public class GeoJsonReaderTest
     public static void onceExecutedBeforeAll()
             throws IOException
     {
-        initialize(GeoJsonReaderTest.class.getName());
-        geoJsonGeomWithFeatureProperty = GeoJsonReaderTest.class.getClassLoader().getResource("testPolygon.json").getPath();
-        geoJsonGeomWithoutFeatureProperty = GeoJsonReaderTest.class.getClassLoader().getResource("testpolygon-no-property.json").getPath();
-        geoJsonWithInvalidGeometries = GeoJsonReaderTest.class.getClassLoader().getResource("testInvalidPolygon.json").getPath();
-        geoJsonWithNullProperty = GeoJsonReaderTest.class.getClassLoader().getResource("testpolygon-with-null-property-value.json").getPath();
-        geoJsonContainsId = GeoJsonReaderTest.class.getClassLoader().getResource("testContainsId.json").getPath();
+        initialize(GeoJsonIOTest.class.getName());
+        geoJsonGeomWithFeatureProperty = GeoJsonIOTest.class.getClassLoader().getResource("testPolygon.json").getPath();
+        geoJsonGeomWithoutFeatureProperty = GeoJsonIOTest.class.getClassLoader().getResource("testpolygon-no-property.json").getPath();
+        geoJsonWithInvalidGeometries = GeoJsonIOTest.class.getClassLoader().getResource("testInvalidPolygon.json").getPath();
+        geoJsonWithNullProperty = GeoJsonIOTest.class.getClassLoader().getResource("testpolygon-with-null-property-value.json").getPath();
+        geoJsonContainsId = GeoJsonIOTest.class.getClassLoader().getResource("testContainsId.json").getPath();
     }
 
     @AfterClass
@@ -74,6 +79,65 @@ public class GeoJsonReaderTest
         assertEquals(geojsonRDD.rawSpatialRDD.count(), 10);
     }
 
+    @Test
+    public void testReadWriteGeoJson()
+            throws IOException
+    {
+        String tmpFilePath = "target/geojson.tmp";
+        SpatialRDD initRdd = GeoJsonReader.readToGeometryRDD(sc, geoJsonGeomWithFeatureProperty);
+        deleteFile(tmpFilePath);
+        initRdd.saveAsGeoJSON(tmpFilePath);
+        SpatialRDD newRdd = GeoJsonReader.readToGeometryRDD(sc, tmpFilePath);
+
+        // Check the basic correctness
+        assertEquals(initRdd.fieldNames.size(), newRdd.fieldNames.size());
+        assertEquals(initRdd.rawSpatialRDD.count(), newRdd.rawSpatialRDD.count());
+
+        // Note that two RDDs may put <field name, value> in different order
+
+        // Put field names and values to a hash map for comparison
+        Geometry initGeom = (Geometry) initRdd.rawSpatialRDD.takeOrdered(1).get(0);
+        String[] initGeomFields = initGeom.getUserData().toString().split("\t");
+        Map<String, Object> initKvs = new HashMap<String, Object>();
+        for (int i = 0; i < initGeomFields.length; i++) {
+            initKvs.put(initRdd.fieldNames.get(i).toString(), initGeomFields[i]);
+        }
+
+        // Put field names and values to a hash map for comparison
+        Geometry newGeom = (Geometry) newRdd.rawSpatialRDD.takeOrdered(1).get(0);
+        String[] newGeomFields = newGeom.getUserData().toString().split("\t");
+        Map<String, Object> newKvs = new HashMap<String, Object>();
+        for (int i = 0; i < initGeomFields.length; i++) {
+            newKvs.put(newRdd.fieldNames.get(i).toString(), newGeomFields[i]);
+        }
+
+        for (int i = 0; i < initRdd.fieldNames.size(); i++) {
+            // The same field name should fetch the same value in both maps
+            assertEquals(initKvs.get(initRdd.fieldNames.get(i).toString()),
+                    newKvs.get(initRdd.fieldNames.get(i).toString()));
+        }
+    }
+
+    @Test
+    public void testReadWriteSpecialGeoJsons()
+            throws IOException
+    {
+        String tmpFilePath = "target/geojson.tmp";
+        SpatialRDD initRdd = GeoJsonReader.readToGeometryRDD(sc, geoJsonGeomWithoutFeatureProperty);
+        deleteFile(tmpFilePath);
+        initRdd.saveAsGeoJSON(tmpFilePath);
+        SpatialRDD newRdd = GeoJsonReader.readToGeometryRDD(sc, tmpFilePath);
+        assertEquals(initRdd.rawSpatialRDD.count(), newRdd.rawSpatialRDD.count());
+
+        initRdd = GeoJsonReader.readToGeometryRDD(sc, geoJsonWithNullProperty);
+        deleteFile(tmpFilePath);
+        initRdd.saveAsGeoJSON(tmpFilePath);
+        newRdd = GeoJsonReader.readToGeometryRDD(sc, tmpFilePath);
+        assertEquals(initRdd.rawSpatialRDD.count(), newRdd.rawSpatialRDD.count());
+
+//        deleteFile(tmpFilePath);
+    }
+
     /**
      * Test geojson with null values in the properties
      *
diff --git a/core/src/test/java/org/apache/sedona/core/formatMapper/TestReadInvalidSyntaxGeometriesTest.java b/core/src/test/java/org/apache/sedona/core/formatMapper/TestReadInvalidSyntaxGeometriesTest.java
index 7b56581a..11d64ae4 100644
--- a/core/src/test/java/org/apache/sedona/core/formatMapper/TestReadInvalidSyntaxGeometriesTest.java
+++ b/core/src/test/java/org/apache/sedona/core/formatMapper/TestReadInvalidSyntaxGeometriesTest.java
@@ -39,7 +39,7 @@ public class TestReadInvalidSyntaxGeometriesTest
     public static void onceExecutedBeforeAll()
             throws IOException
     {
-        initialize(GeoJsonReaderTest.class.getName());
+        initialize(GeoJsonIOTest.class.getName());
         invalidSyntaxGeoJsonGeomWithFeatureProperty = TestReadInvalidSyntaxGeometriesTest.class.getClassLoader().getResource("invalidSyntaxGeometriesJson.json").getPath();
     }