This is an automated email from the ASF dual-hosted git repository. jiayu pushed a commit to branch fix-lonlat-order in repository https://gitbox.apache.org/repos/asf/sedona.git
commit f65bb7ee5b46fc3fb6734e188435dc7f49ab8fc7 Author: Jia Yu <[email protected]> AuthorDate: Tue Aug 8 01:53:42 2023 -0700 Fix the scala code --- .../ScalaEarthdataMapperRunnableExample.scala | 4 +- .../apache/sedona/core/showcase/ScalaExample.scala | 58 +++--------------- .../sedona/core/showcase/SpatialJoinShp.scala | 70 ---------------------- 3 files changed, 11 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala index c1f8f249..4249065c 100644 --- a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala +++ b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala @@ -57,7 +57,7 @@ object ScalaEarthdataMapperRunnableExample extends App { */ def testSpatialRangeQuery() { val earthdataHDFPoint = new EarthdataHDFPointMapper(HDFIncrement, HDFOffset, HDFRootGroupName, HDFDataVariableList, HDFDataVariableName, urlPrefix) - val spatialRDD = new PointRDD(sc, InputLocation, numPartitions, earthdataHDFPoint, StorageLevel.MEMORY_ONLY) + val spatialRDD = new PointRDD(sc, InputLocation, numPartitions, earthdataHDFPoint) var i = 0 while (i < loopTimes) { var resultSize = 0L @@ -71,7 +71,7 @@ object ScalaEarthdataMapperRunnableExample extends App { */ def testSpatialRangeQueryUsingIndex() { val earthdataHDFPoint = new EarthdataHDFPointMapper(HDFIncrement, HDFOffset, HDFRootGroupName, HDFDataVariableList, HDFDataVariableName, urlPrefix) - val spatialRDD = new PointRDD(sc, InputLocation, numPartitions, earthdataHDFPoint, StorageLevel.MEMORY_ONLY) + val spatialRDD = new PointRDD(sc, InputLocation, numPartitions, earthdataHDFPoint) spatialRDD.buildIndex(IndexType.RTREE, false) var i = 0 while (i < loopTimes) { diff --git a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala index ee730fa5..235aa0b6 100644 --- a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala +++ b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala @@ -64,7 +64,7 @@ object ScalaExample extends App { val kNNQueryPoint = geometryFactory.createPoint(new Coordinate(-84.01, 34.01)) val rangeQueryWindow = new Envelope(-90.01, -80.01, 30.01, 40.01) val joinQueryPartitioningType = GridType.QUADTREE - val eachQueryLoopTimes = 5 + val eachQueryLoopTimes = 1 var ShapeFileInputLocation = resourceFolder + "shapefiles/polygon" @@ -76,8 +76,6 @@ object ScalaExample extends App { testSpatialJoinQueryUsingIndex() testDistanceJoinQuery() testDistanceJoinQueryUsingIndex() - testCRSTransformationSpatialRangeQuery() - testCRSTransformationSpatialRangeQueryUsingIndex() sc.stop() System.out.println("All DEMOs passed!") @@ -88,7 +86,7 @@ object ScalaExample extends App { * @throws Exception the exception */ def testSpatialRangeQuery() { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) objectRDD.rawSpatialRDD.persist(StorageLevel.MEMORY_ONLY) for (i <- 1 to eachQueryLoopTimes) { val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, false).count @@ -102,7 +100,7 @@ object ScalaExample extends App { * @throws Exception the exception */ def testSpatialRangeQueryUsingIndex() { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) objectRDD.buildIndex(PointRDDIndexType, false) objectRDD.indexedRawRDD.persist(StorageLevel.MEMORY_ONLY) for (i <- 1 to eachQueryLoopTimes) { @@ -117,7 +115,7 @@ object ScalaExample extends App { * @throws Exception the exception */ def testSpatialKnnQuery() { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) objectRDD.rawSpatialRDD.persist(StorageLevel.MEMORY_ONLY) for (i <- 1 to eachQueryLoopTimes) { val result = KNNQuery.SpatialKnnQuery(objectRDD, kNNQueryPoint, 1000, false) @@ -130,7 +128,7 @@ object ScalaExample extends App { * @throws Exception the exception */ def testSpatialKnnQueryUsingIndex() { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) objectRDD.buildIndex(PointRDDIndexType, false) objectRDD.indexedRawRDD.persist(StorageLevel.MEMORY_ONLY) for (i <- 1 to eachQueryLoopTimes) { @@ -145,7 +143,7 @@ object ScalaExample extends App { */ def testSpatialJoinQuery() { val queryWindowRDD = new PolygonRDD(sc, PolygonRDDInputLocation, PolygonRDDStartOffset, PolygonRDDEndOffset, PolygonRDDSplitter, true) - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) objectRDD.spatialPartitioning(joinQueryPartitioningType) queryWindowRDD.spatialPartitioning(objectRDD.getPartitioner) @@ -164,7 +162,7 @@ object ScalaExample extends App { */ def testSpatialJoinQueryUsingIndex() { val queryWindowRDD = new PolygonRDD(sc, PolygonRDDInputLocation, PolygonRDDStartOffset, PolygonRDDEndOffset, PolygonRDDSplitter, true) - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) objectRDD.spatialPartitioning(joinQueryPartitioningType) queryWindowRDD.spatialPartitioning(objectRDD.getPartitioner) @@ -185,7 +183,7 @@ object ScalaExample extends App { * @throws Exception the exception */ def testDistanceJoinQuery() { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) val queryWindowRDD = new CircleRDD(objectRDD, 0.1) objectRDD.spatialPartitioning(GridType.QUADTREE) @@ -205,7 +203,7 @@ object ScalaExample extends App { * @throws Exception the exception */ def testDistanceJoinQueryUsingIndex() { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY) + val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true) val queryWindowRDD = new CircleRDD(objectRDD, 0.1) objectRDD.spatialPartitioning(GridType.QUADTREE) @@ -221,44 +219,6 @@ object ScalaExample extends App { } } - @throws[Exception] - def testCRSTransformationSpatialRangeQuery(): Unit = { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY, "epsg:4326", "epsg:3005") - objectRDD.rawSpatialRDD.persist(StorageLevel.MEMORY_ONLY) - var i = 0 - while ( { - i < eachQueryLoopTimes - }) { - val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, false).count - assert(resultSize > -1) - - { - i += 1; - i - 1 - } - } - } - - - @throws[Exception] - def testCRSTransformationSpatialRangeQueryUsingIndex(): Unit = { - val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY, "epsg:4326", "epsg:3005") - objectRDD.buildIndex(PointRDDIndexType, false) - objectRDD.indexedRawRDD.persist(StorageLevel.MEMORY_ONLY) - var i = 0 - while ( { - i < eachQueryLoopTimes - }) { - val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, true).count - assert(resultSize > -1) - - { - i += 1; - i - 1 - } - } - } - @throws[Exception] def testLoadShapefileIntoPolygonRDD(): Unit = { val shapefileRDD = new ShapefileRDD(sc, ShapeFileInputLocation) diff --git a/core/src/main/scala/org/apache/sedona/core/showcase/SpatialJoinShp.scala b/core/src/main/scala/org/apache/sedona/core/showcase/SpatialJoinShp.scala deleted file mode 100644 index 00a64cf8..00000000 --- a/core/src/main/scala/org/apache/sedona/core/showcase/SpatialJoinShp.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.sedona.core.showcase - -import org.apache.log4j.{Level, Logger} -import org.apache.sedona.core.formatMapper.shapefileParser.ShapefileRDD -import org.apache.sedona.core.spatialRDD.PolygonRDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.{SparkConf, SparkContext} -import org.locationtech.jts.geom.Polygon - -object SpatialJoinShp extends App { - - val conf = new SparkConf().setAppName("SpatialJoinSpeciesPA").setMaster("local[4]") - - - Logger.getLogger("org").setLevel(Level.WARN) - Logger.getLogger("akka").setLevel(Level.WARN) - val sc = new SparkContext(conf) - val shp1 = new ShapefileRDD(sc, "/Users/jiayu/Downloads/spark4geo_subset/wdpa") - val wdpa = new PolygonRDD(shp1.getPolygonRDD, StorageLevel.MEMORY_ONLY) - val shp2 = new ShapefileRDD(sc, "/Users/jiayu/Downloads/spark4geo_subset/amphib") - val species = new PolygonRDD(shp2.getPolygonRDD, StorageLevel.MEMORY_ONLY) - val result = shp2.getShapeRDD.collect(); - - //wdpa.spatialPartitioning(GridType.QUADTREE) - //species.spatialPartitioning(wdpa.partitionTree) - - def loadShapefile(path: String, numPartitions: Int = 20): PolygonRDD = { - val shp = new ShapefileRDD(sc, path) - val polygon = new PolygonRDD(shp.getPolygonRDD, StorageLevel.MEMORY_ONLY) - //polygon.rawSpatialRDD = polygon.rawSpatialRDD.repartition(numPartitions) - //polygon.analyze() - polygon - } - - for (a <- 1 until result.size()) { - println("print..." + result.get(a).getUserData + " END"); - } - - //val query = JoinQuery.SpatialJoinQuery(wdpa, species, false, false) - - println("polygon is " + shp2.getPolygonRDD.take(100).get(55)) - println("userdata is " + wdpa.rawSpatialRDD.take(100).get(55).asInstanceOf[Polygon].getUserData) - println(species.rawSpatialRDD.count()) - - - //val user_data_sample = JoinQuery.SpatialJoinQuery(wdpa, species, false, false).first()._1.getUserData - //if (user_data_sample.toString.isEmpty) println("UserData is empty") else println(user_data_sample) - - // val join_result = query.rdd.map((tuple: (Polygon, util.HashSet[Polygon])) => (tuple._1, tuple._2.asScala.map(tuple._1.intersection(_).getArea)) ) - // val intersections = join_result.collect() -}
