├── .gitattributes ├── docs ├── image │ ├── logo.png │ ├── favicon.png │ └── school.png ├── fonts │ └── fontawesome-webfont.woff2 └── css │ └── index.css ├── project ├── build.properties └── plugins.sbt ├── src ├── main │ └── scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ └── simba │ │ ├── package.scala │ │ ├── index │ │ ├── package.scala │ │ ├── TreeMapIndex.scala │ │ ├── HashMapIndex.scala │ │ ├── Index.scala │ │ ├── IndexedRelation.scala │ │ ├── HashMapIndexedRelation.scala │ │ ├── TreapIndexedRelation.scala │ │ ├── TreeMapIndexedRelation.scala │ │ ├── QuadTreeIndexedRelation.scala │ │ ├── RTreeIndexedRelation.scala │ │ └── Interval.scala │ │ ├── util │ │ ├── LiteralUtil.scala │ │ ├── NumberUtil.scala │ │ ├── ShapeUtils.scala │ │ ├── PredicateUtil.scala │ │ └── KryoShapeSerializer.scala │ │ ├── execution │ │ ├── SimbaPlan.scala │ │ ├── QueryExecution.scala │ │ ├── join │ │ │ ├── CDJSpark.scala │ │ │ ├── CKJSpark.scala │ │ │ ├── BDJSpark.scala │ │ │ ├── RDJSpark.scala │ │ │ ├── BDJSparkR.scala │ │ │ ├── BKJSpark.scala │ │ │ ├── BKJSparkR.scala │ │ │ ├── DJSpark.scala │ │ │ ├── RKJSpark.scala │ │ │ └── ZKJSpark.scala │ │ └── FilterExec.scala │ │ ├── spatial │ │ ├── Dist.scala │ │ ├── Shape.scala │ │ ├── ZValue.scala │ │ ├── Circle.scala │ │ ├── Point.scala │ │ ├── MBR.scala │ │ ├── Polygon.scala │ │ └── LineSegment.scala │ │ ├── plans │ │ ├── spatialJoinTypes.scala │ │ └── SpatialJoin.scala │ │ ├── ShapeType.scala │ │ ├── examples │ │ ├── SpatialClassInference.scala │ │ ├── BasicSpatialOps.scala │ │ └── IndexExample.scala │ │ ├── expression │ │ ├── PointWrapper.scala │ │ ├── InKNN.scala │ │ ├── InRange.scala │ │ ├── InCircleRange.scala │ │ └── logicalPredicates.scala │ │ ├── partitioner │ │ ├── HashPartitioner.scala │ │ ├── MapDPartitioner.scala │ │ ├── VoronoiPartitioner.scala │ │ ├── RangeDPartitioner.scala │ │ ├── QuadTreePartitioner.scala │ │ └── KDTreePartitioner.scala │ │ ├── ShapeSerializer.scala │ │ └── SimbaOptimizer.scala └── test │ └── scala │ └── org │ └── apache │ └── spark │ └── sql │ └── simba │ ├── spatial │ ├── ZValueSuite.scala │ ├── DistSuite.scala │ ├── ShapeSuite.scala │ ├── CircleSuite.scala │ ├── PointSuite.scala │ ├── PolygonSuite.scala │ ├── MBRSuite.scala │ └── LineSegmentSuite.scala │ └── index │ ├── TreapSuite.scala │ ├── IntervalSuite.scala │ └── RTreeSuite.scala ├── .gitignore └── README.md /.gitattributes: -------------------------------------------------------------------------------- 1 | *.bat text eol=crlf 2 | *.cmd text eol=crlf 3 | -------------------------------------------------------------------------------- /docs/image/logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InitialDLab/Simba/HEAD/docs/image/logo.png -------------------------------------------------------------------------------- /docs/image/favicon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InitialDLab/Simba/HEAD/docs/image/favicon.png -------------------------------------------------------------------------------- /docs/image/school.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InitialDLab/Simba/HEAD/docs/image/school.png -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | // This file should only contain the version of sbt to use. 2 | sbt.version=0.13.8 3 | -------------------------------------------------------------------------------- /docs/fonts/fontawesome-webfont.woff2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InitialDLab/Simba/HEAD/docs/fonts/fontawesome-webfont.woff2 -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql 2 | 3 | /** 4 | * Created by dongx on 3/7/2017. 5 | */ 6 | package object simba { 7 | type DataFrame = Dataset[Row] 8 | } 9 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | // You may use this file to add plugin dependencies for sbt. 2 | resolvers += "bintray-spark-packages" at "https://dl.bintray.com/spark-packages/maven/" 3 | 4 | addSbtPlugin("org.spark-packages" %% "sbt-spark-package" % "0.2.4") 5 | 6 | //addSbtPlugin("com.artima.supersafe" % "sbtplugin" % "1.1.2") 7 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba 18 | 19 | import org.apache.spark.rdd.RDD 20 | 21 | /** 22 | * Created by dongx on 11/12/2016. 23 | */ 24 | package object index { 25 | type IndexedRDD = RDD[IPartition] 26 | } 27 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/util/LiteralUtil.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.util 19 | 20 | import org.apache.spark.sql.simba.ShapeType 21 | import org.apache.spark.sql.simba.spatial.Shape 22 | import org.apache.spark.sql.catalyst.expressions.Literal 23 | 24 | /** 25 | * Created by dongx on 11/14/2016. 26 | */ 27 | object LiteralUtil { 28 | def apply(v: Any): Literal = v match { 29 | case s: Shape => Literal.create(v, ShapeType) 30 | case _ => Literal(v) 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/SimbaPlan.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution 19 | 20 | import org.apache.spark.sql.execution.SparkPlan 21 | import org.apache.spark.sql.simba.SimbaSession 22 | 23 | /** 24 | * Created by dongx on 3/7/17. 25 | */ 26 | abstract class SimbaPlan extends SparkPlan { 27 | 28 | @transient 29 | protected[simba] final val simbaSessionState = SimbaSession.getActiveSession.map(_.sessionState).orNull 30 | 31 | protected override def sparkContext = SimbaSession.getActiveSession.map(_.sparkContext).orNull 32 | } 33 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/Dist.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | /** 21 | * Created by dong on 1/15/16. 22 | * Distance Utilities 23 | */ 24 | object Dist { 25 | def furthest(a: Point, b: MBR) : Double = { 26 | require(a.coord.length == b.low.coord.length) 27 | var ans = 0.0 28 | for (i <- a.coord.indices) { 29 | ans += Math.max((a.coord(i) - b.low.coord(i)) * (a.coord(i) - b.low.coord(i)), 30 | (a.coord(i) - b.high.coord(i)) * (a.coord(i) - b.high.coord(i))) 31 | } 32 | Math.sqrt(ans) 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/TreeMapIndex.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | import org.apache.spark.sql.catalyst.InternalRow 21 | 22 | /** 23 | * Created by dong on 1/15/16. 24 | * Encapsulated TreeMap Index 25 | */ 26 | class TreeMapIndex[T] extends Index with Serializable { 27 | var index = new java.util.TreeMap[T, Int]() 28 | } 29 | 30 | object TreeMapIndex { 31 | def apply[T](data: Array[(T, InternalRow)]): TreeMapIndex[T] = { 32 | val res = new TreeMapIndex[T] 33 | for (i <- data.indices) 34 | res.index.put(data(i)._1, i) 35 | res 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/HashMapIndex.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | import org.apache.spark.sql.catalyst.InternalRow 21 | 22 | /** 23 | * Created by dong on 1/15/16. 24 | * Encapsulated HashMap Index 25 | */ 26 | class HashMapIndex[T] extends Index with Serializable { 27 | var index = new java.util.HashMap[T, Int]() 28 | } 29 | 30 | object HashMapIndex { 31 | def apply[T](data: Array[(T, InternalRow)]): HashMapIndex[T] = { 32 | val res = new HashMapIndex[T] 33 | for (i <- data.indices) { 34 | res.index.put(data(i)._1, i) 35 | } 36 | res 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/ZValueSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | 22 | /** 23 | * Created by Zhihao Bai on 16-7-4. 24 | */ 25 | class ZValueSuite extends FunSuite{ 26 | test("Test for ZValue"){ 27 | assert(ZValue.paddingBinaryBits(7, 8).equals("00000111")) 28 | 29 | var point = Array[Int](2, 0, 1, 6) 30 | 31 | var x = ZValue.apply(point) 32 | assert(x.toBinaryString.equals("110010010")) 33 | 34 | var point1 = ZValue.unapply(x, 4).get 35 | assert(point.length == point1.length) 36 | for(i <- point.indices) 37 | assert(point(i) == point1(i)) 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/Index.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | /** 21 | * Created by dong on 1/15/16. 22 | * Base Traits for index, and definition of index type 23 | */ 24 | trait Index 25 | 26 | object IndexType { 27 | def apply(ty: String): IndexType = ty.toLowerCase match { 28 | case "rtree" => RTreeType 29 | case "treemap" => TreeMapType 30 | case "hashmap" => HashMapType 31 | case "treap" => TreapType 32 | case _ => null 33 | } 34 | } 35 | 36 | sealed abstract class IndexType 37 | 38 | case object RTreeType extends IndexType 39 | case object TreeMapType extends IndexType 40 | case object HashMapType extends IndexType 41 | case object TreapType extends IndexType 42 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/DistSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | 22 | /** 23 | * Created by Zhihao Bai on 16-7-4. 24 | */ 25 | class DistSuite extends FunSuite{ 26 | test("Dist: furthest distance from a point to an MBR"){ 27 | val m = MBR(Point(Array(0.0, 0.0)), Point(Array(2.0, 2.0))) 28 | val p1 = Point(Array(1.0, 1.0)) 29 | val p2 = Point(Array(0.0, 0.0)) 30 | val p3 = Point(Array(1.0, 3.0)) 31 | 32 | assert(Math.abs(Dist.furthest(p1, m) - Math.sqrt(2.0)) < 1e-8) 33 | assert(Math.abs(Dist.furthest(p2, m) - Math.sqrt(8.0)) < 1e-8) 34 | assert(Math.abs(Dist.furthest(p3, m) - Math.sqrt(10.0)) < 1e-8) 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/Shape.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import com.vividsolutions.jts.geom.{Geometry, Polygon => JTSPolygon} 21 | import org.apache.spark.sql.simba.ShapeType 22 | import org.apache.spark.sql.types.SQLUserDefinedType 23 | 24 | /** 25 | * Created by dong on 3/16/16. 26 | */ 27 | @SQLUserDefinedType(udt = classOf[ShapeType]) 28 | abstract class Shape extends Serializable { 29 | def minDist(other: Shape): Double 30 | 31 | def intersects(other: Shape): Boolean 32 | 33 | def getMBR: MBR 34 | 35 | val dimensions: Int 36 | } 37 | 38 | object Shape { 39 | final def apply(g: Geometry): Shape = g match { 40 | case jtsPolygon : JTSPolygon => new Polygon(jtsPolygon) 41 | case _ => null 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/util/NumberUtil.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.util 19 | 20 | import org.apache.spark.sql.catalyst.expressions.Literal 21 | import org.apache.spark.sql.types._ 22 | 23 | /** 24 | * Created by dongx on 11/7/2016. 25 | */ 26 | object NumberUtil { 27 | def literalToDouble(x: Literal): Double = { 28 | x.value match { 29 | case double_value: Number => 30 | double_value.doubleValue() 31 | case decimal_value: Decimal => 32 | decimal_value.toDouble 33 | } 34 | } 35 | 36 | def isIntegral(x: DataType): Boolean = { 37 | x match { 38 | case IntegerType => true 39 | case LongType => true 40 | case ShortType => true 41 | case ByteType => true 42 | case _ => false 43 | } 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *~ 2 | *.#* 3 | *#*# 4 | *.swp 5 | *.ipr 6 | *.iml 7 | *.iws 8 | *.pyc 9 | *.pyo 10 | .credentials 11 | .idea/ 12 | .idea_modules/ 13 | build/*.jar 14 | .settings 15 | .cache 16 | cache 17 | .generated-mima* 18 | work/ 19 | out/ 20 | .DS_Store 21 | third_party/libmesos.so 22 | third_party/libmesos.dylib 23 | build/apache-maven* 24 | build/zinc* 25 | build/scala* 26 | conf/java-opts 27 | conf/*.sh 28 | conf/*.cmd 29 | conf/*.properties 30 | conf/*.conf 31 | conf/*.xml 32 | conf/slaves 33 | docs/_site 34 | docs/api 35 | target/ 36 | reports/ 37 | .project 38 | .classpath 39 | .scala_dependencies 40 | lib_managed/ 41 | src_managed/ 42 | project/boot/ 43 | project/plugins/project/build.properties 44 | project/build/target/ 45 | project/plugins/target/ 46 | project/plugins/lib_managed/ 47 | project/plugins/src_managed/ 48 | logs/ 49 | log/ 50 | spark-tests.log 51 | streaming-tests.log 52 | dependency-reduced-pom.xml 53 | .ensime 54 | .ensime_cache/ 55 | .ensime_lucene 56 | checkpoint 57 | derby.log 58 | dist/ 59 | dev/create-release/*txt 60 | dev/create-release/*final 61 | spark-*-bin-*.tgz 62 | unit-tests.log 63 | /lib/ 64 | ec2/lib/ 65 | rat-results.txt 66 | scalastyle.txt 67 | scalastyle-output.xml 68 | R-unit-tests.log 69 | R/unit-tests.out 70 | python/lib/pyspark.zip 71 | lint-r-report.log 72 | 73 | # For Hive 74 | metastore_db/ 75 | metastore/ 76 | warehouse/ 77 | TempStatsStore/ 78 | sql/hive-thriftserver/test_warehouses 79 | 80 | # For R session data 81 | .RHistory 82 | .RData 83 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/plans/spatialJoinTypes.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.plans 19 | 20 | /** 21 | * Created by dongx on 11/13/2016. 22 | */ 23 | object SpatialJoinType { 24 | def apply(typ: String): SpatialJoinType = typ.toLowerCase.replace("_", "") match { 25 | case "knn" => KNNJoin 26 | case "distance" => DistanceJoin 27 | case "zknn" => ZKNNJoin 28 | case _ => 29 | val supported = Seq("knn", "distance", "zknn") 30 | 31 | throw new IllegalArgumentException(s"Unsupported spatial join type '$typ'. " + 32 | "Supported spatial join types include: " + supported.mkString("'", "', '", "'") + ".") 33 | } 34 | } 35 | 36 | sealed abstract class SpatialJoinType 37 | 38 | case object KNNJoin extends SpatialJoinType 39 | 40 | case object DistanceJoin extends SpatialJoinType 41 | 42 | case object ZKNNJoin extends SpatialJoinType 43 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/ShapeType.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba 19 | 20 | import org.apache.spark.sql.types._ 21 | import org.apache.spark.sql.simba.spatial.Shape 22 | import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayData} 23 | 24 | /** 25 | * Created by dongx on 11/10/16. 26 | */ 27 | private[simba] class ShapeType extends UserDefinedType[Shape] { 28 | override def sqlType: DataType = ArrayType(ByteType, containsNull = false) 29 | 30 | override def serialize(s: Shape): Any = { 31 | new GenericArrayData(ShapeSerializer.serialize(s)) 32 | } 33 | 34 | override def userClass: Class[Shape] = classOf[Shape] 35 | 36 | override def deserialize(datum: Any): Shape = { 37 | datum match { 38 | case values: ArrayData => 39 | ShapeSerializer.deserialize(values.toByteArray) 40 | } 41 | } 42 | } 43 | 44 | case object ShapeType extends ShapeType 45 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/examples/SpatialClassInference.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.examples 19 | 20 | import org.apache.spark.sql.simba.SimbaSession 21 | import org.apache.spark.sql.simba.spatial.Point 22 | 23 | /** 24 | * Created by dongx on 3/16/17. 25 | */ 26 | object SpatialClassInference { 27 | case class PointData(p: Point, payload: Int) 28 | 29 | def main(args: Array[String]) = { 30 | val simbaSession = SimbaSession 31 | .builder() 32 | .master("local[4]") 33 | .appName("SpatialClassInference") 34 | .config("simba.index.partitions", "64") 35 | .getOrCreate() 36 | 37 | import simbaSession.implicits._ 38 | import simbaSession.simbaImplicits._ 39 | val ps = (0 until 10000).map(x => PointData(Point(Array(x.toDouble, x.toDouble)), x + 1)).toDS 40 | ps.knn("p", Array(1.0, 1.0), 4).show() 41 | ps.range("p", Array(1.0, 2.0), Array(4.0, 5.0)).show() 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/expression/PointWrapper.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.expression 19 | 20 | import org.apache.spark.sql.simba.ShapeType 21 | import org.apache.spark.sql.simba.spatial.Point 22 | 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.sql.catalyst.expressions.Expression 25 | import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback 26 | import org.apache.spark.sql.types.DataType 27 | 28 | /** 29 | * Created by dongx on 11/10/16. 30 | */ 31 | case class PointWrapper(exps: Seq[Expression]) 32 | extends Expression with CodegenFallback { 33 | override def nullable: Boolean = false 34 | 35 | override def dataType: DataType = ShapeType 36 | 37 | override def children: Seq[Expression] = exps 38 | 39 | override def eval(input: InternalRow): Any = { 40 | val coord = exps.map(_.eval(input).asInstanceOf[Double]).toArray 41 | Point(coord) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/expression/InKNN.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.expression 19 | 20 | import org.apache.spark.sql.catalyst.InternalRow 21 | import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback 22 | import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, Predicate} 23 | 24 | import org.apache.spark.sql.simba.util.NumberUtil 25 | 26 | /** 27 | * Created by dongx on 11/10/16. 28 | */ 29 | case class InKNN(shape: Expression, target: Expression, k: Literal) 30 | extends Predicate with CodegenFallback { 31 | require(NumberUtil.isIntegral(k.dataType)) 32 | 33 | override def children: Seq[Expression] = Seq(shape, target, k) 34 | 35 | override def nullable: Boolean = false 36 | 37 | override def toString: String = s" **($shape) IN KNN ($target) within ($k)" 38 | 39 | // XX Tricky hack 40 | /** Returns the result of evaluating this expression on a given input Row */ 41 | override def eval(input: InternalRow): Any = true 42 | } 43 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/QueryExecution.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution 19 | 20 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 21 | import org.apache.spark.sql.execution.{SparkPlan, QueryExecution => SQLQueryExecution} 22 | import org.apache.spark.sql.simba.SimbaSession 23 | 24 | /** 25 | * Created by dongx on 3/7/17. 26 | */ 27 | class QueryExecution(val simbaSession: SimbaSession, override val logical: LogicalPlan) 28 | extends SQLQueryExecution(simbaSession, logical) { 29 | 30 | lazy val withIndexedData: LogicalPlan = { 31 | assertAnalyzed() 32 | simbaSession.sessionState.indexManager.useIndexedData(withCachedData) 33 | } 34 | 35 | override lazy val optimizedPlan: LogicalPlan = { 36 | simbaSession.sessionState.optimizer.execute(withIndexedData) 37 | } 38 | 39 | override lazy val sparkPlan: SparkPlan ={ 40 | SimbaSession.setActiveSession(simbaSession) 41 | simbaSession.sessionState.planner.plan(optimizedPlan).next() 42 | } 43 | 44 | } 45 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/partitioner/HashPartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.partitioner 19 | 20 | import org.apache.spark.{Partitioner, SparkEnv} 21 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 22 | import org.apache.spark.shuffle.sort.SortShuffleManager 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.util.MutablePair 25 | 26 | /** 27 | * Created by dong on 1/15/16. 28 | * Linear Hash Partitioner with Java hashcode 29 | */ 30 | object HashPartition { 31 | def sortBasedShuffleOn: Boolean = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] 32 | 33 | def apply(origin: RDD[(Any, InternalRow)], num_partitions: Int): RDD[(Any, InternalRow)] = { 34 | val rdd = if (sortBasedShuffleOn) { 35 | origin.mapPartitions {iter => iter.map(row => (row._1, row._2.copy()))} 36 | } else { 37 | origin.mapPartitions {iter => 38 | val mutablePair = new MutablePair[Any, InternalRow]() 39 | iter.map(row => mutablePair.update(row._1, row._2.copy())) 40 | } 41 | } 42 | 43 | val part = new HashPartitioner(num_partitions) 44 | new ShuffledRDD[Any, InternalRow, InternalRow](rdd, part) 45 | } 46 | } 47 | 48 | class HashPartitioner(num_partitions: Int) extends Partitioner { 49 | override def numPartitions: Int = num_partitions 50 | 51 | override def getPartition(key: Any): Int = { 52 | key.hashCode() % num_partitions 53 | } 54 | } -------------------------------------------------------------------------------- /docs/css/index.css: -------------------------------------------------------------------------------- 1 | .html{ 2 | font-family: Helvetica,Arial,sans-serif; 3 | } 4 | 5 | .get { 6 | background: #eee; 7 | color: #333; 8 | text-align: left; 9 | padding-top: 30px; 10 | padding-bottom: 60px; 11 | padding-left: 100px; 12 | padding-right: 100px; 13 | } 14 | 15 | .get-title { 16 | background: url(../image/logo.png); 17 | background-repeat: no-repeat; 18 | background-position: left, top; 19 | font-size: 400%; 20 | padding-bottom: 20px; 21 | padding-top: 60px; 22 | display: inline-block; 23 | } 24 | 25 | .get-sub-title { 26 | font-size: 20pt; 27 | padding-left: 290px; 28 | color: #888; 29 | } 30 | 31 | .index-intro { 32 | font-size: 15pt; 33 | padding-left: 40px; 34 | } 35 | 36 | .get-btn { 37 | background: #fff; 38 | } 39 | 40 | .detail { 41 | background: #fff; 42 | } 43 | 44 | .detail-h2 { 45 | text-align: center; 46 | font-size: 180%; 47 | margin: 30px 0; 48 | } 49 | 50 | .detail-h3 { 51 | color: #1f8dd6; 52 | } 53 | 54 | .detail-p { 55 | color: #7f8c8d; 56 | } 57 | 58 | .detail-mb { 59 | margin-bottom: 30px; 60 | } 61 | 62 | .publication { 63 | padding-left: 120px; 64 | padding-right: 100px; 65 | padding-bottom: 30px; 66 | } 67 | .team { 68 | padding-left: 120px; 69 | padding-right: 100px; 70 | padding-bottom: 30px; 71 | } 72 | 73 | .faq { 74 | padding-left: 120px; 75 | padding-right: 100px; 76 | padding-bottom: 30px; 77 | 78 | } 79 | 80 | .news { 81 | padding-left: 120px; 82 | padding-right: 100px; 83 | padding-top: 20px; 84 | font-size: 10pt; 85 | color: #3a87ad; 86 | } 87 | 88 | .example { 89 | padding-left: 120px; 90 | padding-right: 100px; 91 | padding-top: 40px; 92 | } 93 | 94 | .footer p { 95 | color: #7f8c8d; 96 | margin: 0; 97 | padding: 15px 0; 98 | text-align: center; 99 | background: #2d3e50; 100 | } 101 | 102 | .text-bold { 103 | font-weight: bold; 104 | } 105 | 106 | .text-bule { 107 | color: #317eac; 108 | } 109 | .text-big { 110 | font-size: 200%; 111 | } 112 | .text-small { 113 | font-size: 90%; 114 | } 115 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/expression/InRange.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.expression 19 | 20 | import org.apache.spark.sql.simba.{ShapeSerializer, ShapeType} 21 | import org.apache.spark.sql.catalyst.InternalRow 22 | import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, Predicate} 23 | import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback 24 | import org.apache.spark.sql.simba.spatial.{MBR, Point, Shape} 25 | import org.apache.spark.sql.simba.util.ShapeUtils 26 | import org.apache.spark.sql.catalyst.util.GenericArrayData 27 | 28 | /** 29 | * Created by dongx on 11/10/16. 30 | */ 31 | case class InRange(shape: Expression, range_low: Expression, range_high: Expression) 32 | extends Predicate with CodegenFallback{ 33 | override def nullable: Boolean = false 34 | 35 | override def eval(input: InternalRow): Any = { 36 | val eval_shape = ShapeUtils.getShape(shape, input) 37 | val eval_low = range_low.asInstanceOf[Literal].value.asInstanceOf[Point] 38 | val eval_high = range_high.asInstanceOf[Literal].value.asInstanceOf[Point] 39 | require(eval_shape.dimensions == eval_low.dimensions && eval_shape.dimensions == eval_high.dimensions) 40 | val mbr = MBR(eval_low, eval_high) 41 | mbr.intersects(eval_shape) 42 | } 43 | 44 | override def toString: String = s" **($shape) IN Rectangle ($range_low) - ($range_high)** " 45 | 46 | override def children: Seq[Expression] = Seq(shape, range_low, range_high) 47 | } 48 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/ShapeSerializer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba 19 | 20 | import com.esotericsoftware.kryo.Kryo 21 | import com.esotericsoftware.kryo.io.Input 22 | import com.esotericsoftware.kryo.io.Output 23 | import java.io._ 24 | 25 | import org.apache.spark.sql.simba.spatial._ 26 | import org.apache.spark.sql.simba.util.KryoShapeSerializer 27 | 28 | /** 29 | * Created by dongx on 11/15/16. 30 | */ 31 | object ShapeSerializer { 32 | private[simba] val kryo = new Kryo() 33 | 34 | kryo.register(classOf[Shape], new KryoShapeSerializer) 35 | kryo.register(classOf[Point], new KryoShapeSerializer) 36 | kryo.register(classOf[MBR], new KryoShapeSerializer) 37 | kryo.register(classOf[Polygon], new KryoShapeSerializer) 38 | kryo.register(classOf[Circle], new KryoShapeSerializer) 39 | kryo.register(classOf[LineSegment], new KryoShapeSerializer) 40 | kryo.addDefaultSerializer(classOf[Shape], new KryoShapeSerializer) 41 | kryo.setReferences(false) 42 | 43 | def deserialize(data: Array[Byte]): Shape = { 44 | val in = new ByteArrayInputStream(data) 45 | val input = new Input(in) 46 | val res = kryo.readObject(input, classOf[Shape]) 47 | input.close() 48 | res 49 | } 50 | 51 | def serialize(o: Shape): Array[Byte] = { 52 | val out = new ByteArrayOutputStream() 53 | val output = new Output(out) 54 | kryo.writeObject(output, o) 55 | output.close() 56 | out.toByteArray 57 | } 58 | } 59 | 60 | class ShapeSerializer { 61 | 62 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/examples/BasicSpatialOps.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.simba.examples 2 | 3 | import org.apache.spark.sql.simba.SimbaSession 4 | 5 | /** 6 | * Created by dongx on 3/7/2017. 7 | */ 8 | object BasicSpatialOps { 9 | case class PointData(x: Double, y: Double, z: Double, other: String) 10 | 11 | def main(args: Array[String]): Unit = { 12 | 13 | val simbaSession = SimbaSession 14 | .builder() 15 | .master("local[4]") 16 | .appName("SparkSessionForSimba") 17 | .config("simba.join.partitions", "20") 18 | .getOrCreate() 19 | 20 | runRangeQuery(simbaSession) 21 | runKnnQuery(simbaSession) 22 | runJoinQUery(simbaSession) 23 | simbaSession.stop() 24 | } 25 | 26 | private def runKnnQuery(simba: SimbaSession): Unit = { 27 | 28 | import simba.implicits._ 29 | val caseClassDS = Seq(PointData(1.0, 1.0, 3.0, "1"), PointData(2.0, 2.0, 3.0, "2"), PointData(2.0, 2.0, 3.0, "3"), 30 | PointData(2.0, 2.0, 3.0, "4"),PointData(3.0, 3.0, 3.0, "5"),PointData(4.0, 4.0, 3.0, "6")).toDS() 31 | 32 | import simba.simbaImplicits._ 33 | caseClassDS.knn(Array("x", "y"),Array(1.0, 1.0),4).show(4) 34 | 35 | } 36 | 37 | private def runRangeQuery(simba: SimbaSession): Unit = { 38 | 39 | import simba.implicits._ 40 | val caseClassDS = Seq(PointData(1.0, 1.0, 3.0, "1"), PointData(2.0, 2.0, 3.0, "2"), PointData(2.0, 2.0, 3.0, "3"), 41 | PointData(2.0, 2.0, 3.0, "4"),PointData(3.0, 3.0, 3.0, "5"),PointData(4.0, 4.0, 3.0, "6")).toDS() 42 | 43 | import simba.simbaImplicits._ 44 | caseClassDS.range(Array("x", "y"),Array(1.0, 1.0),Array(3.0, 3.0)).show(10) 45 | 46 | } 47 | 48 | private def runJoinQUery(simba: SimbaSession): Unit = { 49 | 50 | import simba.implicits._ 51 | 52 | val DS1 = (0 until 10000).map(x => PointData(x, x + 1, x + 2, x.toString)).toDS 53 | val DS2 = (0 until 10000).map(x => PointData(x, x, x + 1, x.toString)).toDS 54 | 55 | import simba.simbaImplicits._ 56 | 57 | DS1.knnJoin(DS2, Array("x", "y"),Array("x", "y"), 3).show() 58 | 59 | DS1.distanceJoin(DS2, Array("x", "y"),Array("x", "y"), 3).show() 60 | 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/ShapeSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | import com.vividsolutions.jts.geom.{GeometryFactory, Polygon => JTSPolygon} 22 | import com.vividsolutions.jts.io.WKTReader 23 | 24 | /** 25 | * Created by Zhihao Bai on 16-7-5. 26 | */ 27 | class ShapeSuite extends FunSuite{ 28 | def intersects(s1: Shape, s2: Shape): Boolean = { 29 | s1.intersects(s2) 30 | } 31 | 32 | def minDist(s1: Shape, s2: Shape): Double = { 33 | s1.minDist(s2) 34 | } 35 | 36 | test("Shape: Abstract Functions"){ 37 | val p = Point(Array(-1.0, 0.0)) 38 | val s = LineSegment(Point(Array(-1.0, 0.0)), Point(Array(1.0, 1.0))) 39 | val m = MBR(Point(Array(0.0, 0.0)), Point(Array(2.0, 2.0))) 40 | val c = Circle(Point(Array(0.0, -1.0)), 1.0) 41 | 42 | assert(!intersects(p, c)) 43 | assert(intersects(s, m)) 44 | 45 | assert(Math.abs(minDist(p, c) - (Math.sqrt(2.0) - 1.0)) < 1e-8) 46 | assert(Math.abs(minDist(s, m)) < 1e-8) 47 | } 48 | 49 | test("Shape: apply Geometry"){ 50 | val gf = new GeometryFactory() 51 | val reader = new WKTReader( gf ) 52 | 53 | val point = reader.read("POINT (0.0 0.0)") 54 | assert(Shape.apply(point) == null) 55 | 56 | val ply = Shape.apply(reader.read("POLYGON((2.0 1.0, 3.0 0.0, 4.0 1.0, 3.0 2.0, 2.0 1.0))")) 57 | assert(ply != null) 58 | val p = Point(Array(2.0, 0.0)) 59 | assert(Math.abs(ply.minDist(p) - Math.sqrt(2.0) / 2.0) < 1e-8) 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/ZValue.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | /** 21 | * Created by dong on 1/15/16. 22 | * Utilities for Z-Value Curves 23 | */ 24 | object ZValue { 25 | def paddingBinaryBits(source: Int, digits: Int): String = { 26 | val pd_length = digits - source.toBinaryString.length 27 | "0" * pd_length + source.toBinaryString 28 | } 29 | 30 | // TODO shift Long to BitInt for supporting bigger Z-Values 31 | def apply(point: Array[Int]): Long = { 32 | var maxBit = 0 33 | for (i <- point.indices) 34 | if (point(i).toBinaryString.length > maxBit) { 35 | maxBit = point(i).toBinaryString.length 36 | } 37 | 38 | var ans = "" 39 | val pointStrs = point.map(x => paddingBinaryBits(x, maxBit)) 40 | 41 | for (i <- 0 until maxBit) 42 | for (j <- point.indices) 43 | ans += pointStrs(j)(i) 44 | 45 | java.lang.Long.parseLong(ans, 2) 46 | } 47 | 48 | def unapply(value: Long, dimension: Int): Option[Array[Int]] = { 49 | val ans = new Array[Int](dimension) 50 | val binaryZValue = value.toBinaryString 51 | var currentBit = binaryZValue.length - 1 52 | var shiftBase = 1 53 | while (currentBit >= 0) { 54 | for (i <- 0 until dimension) 55 | if (currentBit - dimension + 1 + i >= 0) { 56 | ans(i) += shiftBase * binaryZValue(currentBit - dimension + 1 + i).toString.toInt 57 | } 58 | 59 | currentBit -= dimension 60 | shiftBase *= 2 61 | } 62 | Some(ans) 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/partitioner/MapDPartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.partitioner 19 | 20 | import org.apache.spark.{Partitioner, SparkEnv} 21 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 22 | import org.apache.spark.shuffle.sort.SortShuffleManager 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.util.MutablePair 25 | 26 | /** 27 | * Created by dong on 1/15/16. 28 | * Determined Key-Mapping Partitioner 29 | */ 30 | object MapDPartition { 31 | def sortBasedShuffleOn: Boolean = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] 32 | 33 | def apply[T](origin: RDD[(Int, (T, InternalRow))], 34 | num_partitions: Int): RDD[(Int, (T, InternalRow))] = { 35 | val rdd = if (sortBasedShuffleOn) { 36 | origin.mapPartitions {iter => iter.map(row => (row._1, (row._2._1, row._2._2.copy())))} 37 | } else { 38 | origin.mapPartitions {iter => 39 | val mutablePair = new MutablePair[Int, (T, InternalRow)]() 40 | iter.map(row => mutablePair.update(row._1, (row._2._1, row._2._2.copy()))) 41 | } 42 | } 43 | 44 | val part = new MapDPartitioner(num_partitions) 45 | new ShuffledRDD[Int, (T, InternalRow), (T, InternalRow)](rdd, part) 46 | } 47 | } 48 | 49 | class MapDPartitioner(num_partitions: Int) extends Partitioner { 50 | def numPartitions: Int = num_partitions 51 | def getPartition(key: Any): Int = { 52 | val k = key.asInstanceOf[Int] 53 | require(k >= 0 && k < num_partitions) 54 | k 55 | } 56 | } 57 | 58 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/plans/SpatialJoin.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.plans 19 | 20 | import org.apache.spark.sql.simba.expression.{InCircleRange, InKNN} 21 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} 22 | import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, LogicalPlan} 23 | import org.apache.spark.sql.types.BooleanType 24 | 25 | /** 26 | * Created by dongx on 11/13/2016. 27 | */ 28 | case class SpatialJoin(left: LogicalPlan, right: LogicalPlan, joinType: SpatialJoinType, 29 | condition: Option[Expression]) extends BinaryNode { 30 | override def output: Seq[Attribute] = { 31 | joinType match { 32 | case KNNJoin => 33 | require(condition.get.isInstanceOf[InKNN]) 34 | left.output ++ right.output 35 | case ZKNNJoin => 36 | require(condition.get.isInstanceOf[InKNN]) 37 | left.output ++ right.output 38 | case DistanceJoin => 39 | require(condition.get.isInstanceOf[InCircleRange]) 40 | left.output ++ right.output.map(_.withNullability(true)) 41 | case _ => 42 | left.output ++ right.output 43 | } 44 | } 45 | 46 | def selfJoinResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty 47 | 48 | // Joins are only resolved if they don't introduce ambiguous expression ids. 49 | override lazy val resolved: Boolean = { 50 | childrenResolved && 51 | expressions.forall(_.resolved) && 52 | selfJoinResolved && 53 | condition.forall(_.dataType == BooleanType) 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/expression/InCircleRange.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.expression 19 | 20 | import org.apache.spark.sql.simba.{ShapeSerializer, ShapeType} 21 | import org.apache.spark.sql.catalyst.InternalRow 22 | import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback 23 | import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, Predicate} 24 | import org.apache.spark.sql.types.NumericType 25 | import org.apache.spark.sql.simba.spatial.{Circle, Point, Shape} 26 | import org.apache.spark.sql.simba.util.{NumberUtil, ShapeUtils} 27 | import org.apache.spark.sql.catalyst.util.GenericArrayData 28 | 29 | /** 30 | * Created by dongx on 11/10/16. 31 | */ 32 | case class InCircleRange(shape: Expression, target: Expression, radius: Literal) 33 | extends Predicate with CodegenFallback { 34 | require(radius.dataType.isInstanceOf[NumericType]) 35 | 36 | override def children: Seq[Expression] = Seq(shape, target, radius) 37 | 38 | override def nullable: Boolean = false 39 | 40 | override def toString: String = s" **($shape) IN CIRCLERANGE ($target) within ($radius)** " 41 | 42 | /** Returns the result of evaluating this expression on a given input Row */ 43 | override def eval(input: InternalRow): Any = { 44 | val eval_shape = ShapeUtils.getShape(shape, input) 45 | val eval_target = target.eval(input).asInstanceOf[Point] 46 | require(eval_shape.dimensions == eval_target.dimensions) 47 | val eval_r = NumberUtil.literalToDouble(radius) 48 | Circle(eval_target, eval_r).intersects(eval_shape) 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/CDJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.spatial.Point 21 | import org.apache.spark.sql.simba.util.{NumberUtil, ShapeUtils} 22 | import org.apache.spark.rdd.RDD 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 25 | import org.apache.spark.sql.catalyst.plans.physical.Partitioning 26 | import org.apache.spark.sql.execution.SparkPlan 27 | 28 | /** 29 | * Created by dong on 1/20/16. 30 | * Distance Join based on Cartesian Product 31 | */ 32 | case class CDJSpark(left_key: Expression, right_key: Expression, 33 | l: Literal, left: SparkPlan, right: SparkPlan) extends SparkPlan { 34 | override def outputPartitioning: Partitioning = left.outputPartitioning 35 | 36 | override def output: Seq[Attribute] = left.output ++ right.output 37 | 38 | final val r = NumberUtil.literalToDouble(l) 39 | 40 | override protected def doExecute(): RDD[InternalRow] = 41 | left.execute().cartesian(right.execute()).mapPartitions { iter => 42 | val joinedRow = new JoinedRow 43 | iter.filter { row => 44 | val point1 = ShapeUtils.getShape(left_key, left.output, row._1).asInstanceOf[Point] 45 | val point2 = ShapeUtils.getShape(right_key, right.output, row._2).asInstanceOf[Point] 46 | point1.minDist(point2) <= r 47 | }.map(row => joinedRow(row._1, row._2)) 48 | } 49 | 50 | override def children: Seq[SparkPlan] = Seq(left, right) 51 | } 52 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/partitioner/VoronoiPartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.partitioner 19 | 20 | import org.apache.spark.sql.simba.spatial.Point 21 | import org.apache.spark.{Partitioner, SparkEnv} 22 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 23 | import org.apache.spark.shuffle.sort.SortShuffleManager 24 | import org.apache.spark.sql.catalyst.InternalRow 25 | import org.apache.spark.util.MutablePair 26 | 27 | /** 28 | * Created by dong on 1/20/16. 29 | * Voronoi Partitioner which assigns points to its nearest pivot 30 | */ 31 | object VoronoiPartition { 32 | def sortBasedShuffleOn: Boolean = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] 33 | 34 | def apply(origin: RDD[(Int, (Point, InternalRow))], pivot_to_group: Array[Int], num_group: Int) 35 | : RDD[(Int, (Point, InternalRow))] = { 36 | val rdd = if (sortBasedShuffleOn) { 37 | origin.mapPartitions {iter => iter.map(row => (row._1, (row._2._1, row._2._2.copy())))} 38 | } else { 39 | origin.mapPartitions {iter => 40 | val mutablePair = new MutablePair[Int, (Point, InternalRow)]() 41 | iter.map(row => mutablePair.update(row._1, (row._2._1, row._2._2.copy()))) 42 | } 43 | } 44 | 45 | val part = new VoronoiPartitioner(pivot_to_group, num_group) 46 | new ShuffledRDD[Int, (Point, InternalRow), (Point, InternalRow)](rdd, part) 47 | } 48 | } 49 | 50 | class VoronoiPartitioner(pivot_to_group: Array[Int], num_group: Int) extends Partitioner { 51 | override def numPartitions: Int = num_group 52 | 53 | override def getPartition(key: Any): Int = { 54 | val k = key.asInstanceOf[Int] 55 | pivot_to_group(k) 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/IndexedRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba.index 18 | 19 | import org.apache.spark.sql.simba.SimbaSession 20 | import org.apache.spark.sql.catalyst.InternalRow 21 | import org.apache.spark.sql.catalyst.expressions.Attribute 22 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 23 | import org.apache.spark.sql.execution.SparkPlan 24 | 25 | /** 26 | * Created by dong on 1/15/16. 27 | * Indexed Relation Structures for Simba 28 | */ 29 | 30 | private[simba] case class IPartition(data: Array[InternalRow], index: Index) 31 | 32 | private[simba] object IndexedRelation { 33 | def apply(child: SparkPlan, table_name: Option[String], index_type: IndexType, 34 | column_keys: List[Attribute], index_name: String): IndexedRelation = { 35 | index_type match { 36 | case TreeMapType => 37 | TreeMapIndexedRelation(child.output, child, table_name, column_keys, index_name)() 38 | case TreapType => 39 | TreapIndexedRelation(child.output, child, table_name, column_keys, index_name)() 40 | case RTreeType => 41 | RTreeIndexedRelation(child.output, child, table_name, column_keys, index_name)() 42 | case HashMapType => 43 | HashMapIndexedRelation(child.output, child, table_name, column_keys, index_name)() 44 | case _ => null 45 | } 46 | } 47 | } 48 | 49 | private[simba] abstract class IndexedRelation extends LogicalPlan { 50 | self: Product => 51 | var _indexedRDD: IndexedRDD 52 | def indexedRDD: IndexedRDD = _indexedRDD 53 | 54 | def simbaSession = SimbaSession.getActiveSession.orNull 55 | 56 | override def children: Seq[LogicalPlan] = Nil 57 | def output: Seq[Attribute] 58 | 59 | def withOutput(newOutput: Seq[Attribute]): IndexedRelation 60 | } 61 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/index/TreapSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | import org.apache.spark.sql.catalyst.InternalRow 21 | import org.scalatest.FunSuite 22 | 23 | /** 24 | * Created by Zhihao Bai on 16-7-19. 25 | */ 26 | class TreapSuite extends FunSuite{ 27 | val data = new Array[(Int, InternalRow)](101) 28 | val divisor = 11 29 | for(i <- 0 to 100) { 30 | data(i) = (i % divisor, null) 31 | } 32 | 33 | val treap = Treap.apply[Int](data) 34 | 35 | test("Treap: rank"){ 36 | var rank = 0 37 | var count = 0 38 | for(i <- 0 to 10) { 39 | count = 101 / divisor 40 | if(i == 0 || i == 1) count = count + 1 41 | rank += count 42 | assert(treap.rank(i) == rank) 43 | } 44 | } 45 | 46 | test("Treap: getCount"){ 47 | var count = 0 48 | for(i <- 0 to 10) { 49 | count = 101 / divisor 50 | if(i == 0 || i == 1) count = count + 1 51 | assert(treap.getCount(i) == count) 52 | } 53 | } 54 | 55 | test("Treap: find"){ 56 | var count = 0 57 | for(i <- 0 to 10) { 58 | count = 101 / divisor 59 | if(i == 0 || i == 1) count = count + 1 60 | var ints = treap.find(i) 61 | assert(ints.length == count) 62 | for(j <- ints) { 63 | assert(j % divisor == i) 64 | } 65 | } 66 | } 67 | 68 | test("Treap: range, simple"){ 69 | var ints = treap.range(3, 5) 70 | for(j <- ints){ 71 | assert(j % divisor >= 3 && j % divisor <= 5) 72 | } 73 | } 74 | test("Treap: range, complex"){ 75 | val ints = treap.range(3, 5, 5, 1.1, isNumeric = true).get 76 | for(j <- ints){ 77 | assert(j % divisor >= 3 && j % divisor <= 5) 78 | } 79 | 80 | val ints_none = treap.range(0, 10, 1, 0.0, isNumeric = true) 81 | assert(ints_none.isEmpty) 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/CKJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.spatial.Point 22 | import org.apache.spark.sql.simba.util.ShapeUtils 23 | import org.apache.spark.rdd.RDD 24 | import org.apache.spark.sql.catalyst.InternalRow 25 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 26 | import org.apache.spark.sql.catalyst.plans.physical.Partitioning 27 | import org.apache.spark.sql.execution.SparkPlan 28 | 29 | /** 30 | * Created by dong on 1/20/16. 31 | * KNN Join based on Cartesian Product 32 | */ 33 | case class CKJSpark(left_key: Expression, right_key: Expression, 34 | l: Literal, left: SparkPlan, right: SparkPlan) extends SimbaPlan { 35 | override def outputPartitioning: Partitioning = left.outputPartitioning 36 | 37 | override def output: Seq[Attribute] = left.output ++ right.output 38 | 39 | final val k = l.value.asInstanceOf[Number].intValue() 40 | 41 | override protected def doExecute(): RDD[InternalRow] = { 42 | val left_rdd = left.execute() 43 | val right_rdd = right.execute() 44 | 45 | left_rdd.map(row => 46 | (ShapeUtils.getShape(left_key, left.output, row).asInstanceOf[Point], row) 47 | ).cartesian(right_rdd).map { 48 | case (l: (Point, InternalRow), r: InternalRow) => 49 | val tmp_point = ShapeUtils.getShape(right_key, right.output, r).asInstanceOf[Point] 50 | l._2 -> List((tmp_point.minDist(l._1), r)) 51 | }.reduceByKey { 52 | case (l_list: Seq[(Double, InternalRow)], r_list: Seq[(Double, InternalRow)]) => 53 | (l_list ++ r_list).sortWith(_._1 < _._1).take(k) 54 | }.flatMapValues(list => list).mapPartitions { iter => 55 | val joinedRow = new JoinedRow 56 | iter.map(r => joinedRow(r._1, r._2._2)) 57 | } 58 | } 59 | 60 | override def children: Seq[SparkPlan] = Seq(left, right) 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/Circle.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.apache.spark.sql.simba.ShapeType 21 | import org.apache.spark.sql.types.SQLUserDefinedType 22 | 23 | 24 | /** 25 | * Created by dong on 3/16/16. 26 | */ 27 | @SQLUserDefinedType(udt = classOf[ShapeType]) 28 | case class Circle(center: Point, radius: Double) extends Shape { 29 | override val dimensions: Int = center.dimensions 30 | 31 | override def intersects(other: Shape): Boolean = { 32 | other match { 33 | case p: Point => contains(p) 34 | case mbr: MBR => intersects(mbr) 35 | case cir: Circle => intersects(cir) 36 | case poly: Polygon => poly.intersects(this) 37 | case seg: LineSegment => seg.intersects(this) 38 | } 39 | } 40 | 41 | override def minDist(other: Shape): Double = { 42 | other match { 43 | case p: Point => minDist(p) 44 | case mbr: MBR => minDist(mbr) 45 | case cir: Circle => minDist(cir) 46 | case poly: Polygon => poly.minDist(this) 47 | case seg: LineSegment => seg.minDist(this) 48 | } 49 | } 50 | 51 | def minDist(other: Point): Double = { 52 | require(center.coord.length == other.coord.length) 53 | if (contains(other)) 0.0 54 | else other.minDist(center) - radius 55 | } 56 | 57 | def minDist(other: MBR): Double = { 58 | require(center.coord.length == other.low.coord.length) 59 | if (intersects(other)) 0.0 60 | else center.minDist(other) - radius 61 | } 62 | 63 | def minDist(other: Circle): Double = { 64 | require(center.coord.length == other.center.coord.length) 65 | if (intersects(other)) 0.0 66 | else center.minDist(other.center) - radius - other.radius 67 | } 68 | 69 | def contains(p: Point): Boolean = p.minDist(center) <= radius 70 | 71 | def intersects(other: MBR): Boolean = center.minDist(other) <= radius 72 | 73 | def intersects(other: Circle): Boolean = other.center.minDist(center) <= other.radius + radius 74 | 75 | def getMBR: MBR = MBR(center.shift(-radius), center.shift(radius)) 76 | 77 | override def toString: String = "CIRCLE(" + center.toString + "," + radius + ")" 78 | } 79 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/HashMapIndexedRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba.index 18 | 19 | import org.apache.spark.sql.simba.partitioner.HashPartition 20 | import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation 21 | import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences} 22 | import org.apache.spark.sql.execution.SparkPlan 23 | import org.apache.spark.sql.types.NumericType 24 | import org.apache.spark.storage.StorageLevel 25 | 26 | /** 27 | * Created by dongx on 11/12/2016. 28 | */ 29 | private[simba] case class HashMapIndexedRelation(output: Seq[Attribute], child: SparkPlan, 30 | table_name: Option[String], column_keys: List[Attribute], index_name: String)(var _indexedRDD: IndexedRDD = null) 31 | extends IndexedRelation with MultiInstanceRelation { 32 | 33 | require(column_keys.length == 1) 34 | require(column_keys.head.dataType.isInstanceOf[NumericType]) 35 | if (_indexedRDD == null) { 36 | buildIndex() 37 | } 38 | 39 | private[simba] def buildIndex(): Unit = { 40 | val numShufflePartitions = simbaSession.sessionState.simbaConf.indexPartitions 41 | 42 | val dataRDD = child.execute().map(row => { 43 | val eval_key = BindReferences.bindReference(column_keys.head, child.output).eval(row) 44 | (eval_key, row) 45 | }) 46 | 47 | val partitionedRDD = HashPartition(dataRDD, numShufflePartitions) 48 | val indexed = partitionedRDD.mapPartitions(iter => { 49 | val data = iter.toArray 50 | val index = HashMapIndex(data) 51 | Array(IPartition(data.map(_._2), index)).iterator 52 | }).persist(StorageLevel.MEMORY_AND_DISK_SER) 53 | 54 | indexed.setName(table_name.map(n => s"$n $index_name").getOrElse(child.toString)) 55 | _indexedRDD = indexed 56 | } 57 | 58 | override def newInstance(): IndexedRelation = { 59 | HashMapIndexedRelation(output.map(_.newInstance()), child, table_name, 60 | column_keys, index_name)(_indexedRDD).asInstanceOf[this.type] 61 | } 62 | 63 | override def withOutput(new_output: Seq[Attribute]): IndexedRelation = { 64 | HashMapIndexedRelation(new_output, child, table_name, column_keys, index_name)(_indexedRDD) 65 | } 66 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/partitioner/RangeDPartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.partitioner 19 | 20 | import org.apache.spark.util.CollectionsUtils 21 | import org.apache.spark.{Partitioner, SparkEnv} 22 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 23 | import org.apache.spark.shuffle.sort.SortShuffleManager 24 | import org.apache.spark.sql.catalyst.InternalRow 25 | import org.apache.spark.util.MutablePair 26 | 27 | import scala.reflect.ClassTag 28 | 29 | /** 30 | * Created by dong on 1/15/16. 31 | * Range Partitoner with Determined Range Bounds 32 | */ 33 | object RangeDPartition { 34 | def sortBasedShuffleOn: Boolean = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] 35 | 36 | def apply[K: Ordering: ClassTag, T](origin: RDD[(K, (T, InternalRow))], 37 | range_bounds: Array[K]): RDD[(K, (T, InternalRow))] = { 38 | val rdd = if (sortBasedShuffleOn) { 39 | origin.mapPartitions {iter => iter.map(row => (row._1, (row._2._1, row._2._2.copy())))} 40 | } else { 41 | origin.mapPartitions {iter => 42 | val mutablePair = new MutablePair[K, (T, InternalRow)]() 43 | iter.map(row => mutablePair.update(row._1, (row._2._1, row._2._2.copy()))) 44 | } 45 | } 46 | 47 | val part = new RangeDPartitioner(range_bounds, ascending = true) 48 | new ShuffledRDD[K, (T, InternalRow), (T, InternalRow)](rdd, part) 49 | } 50 | } 51 | 52 | class RangeDPartitioner[K: Ordering: ClassTag](range_bounds: Array[K], 53 | ascending: Boolean) extends Partitioner { 54 | def numPartitions: Int = range_bounds.length + 1 55 | 56 | private val binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] 57 | 58 | def getPartition(key: Any): Int = { 59 | val k = key.asInstanceOf[K] 60 | var partition = 0 61 | if (range_bounds.length < 128) { 62 | while (partition < range_bounds.length && Ordering[K].gt(k, range_bounds(partition))) 63 | partition += 1 64 | } else { 65 | partition = binarySearch(range_bounds, k) 66 | if (partition < 0) partition = -partition - 1 67 | if (partition > range_bounds.length) partition = range_bounds.length 68 | } 69 | if (ascending) partition 70 | else range_bounds.length - partition 71 | } 72 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/TreapIndexedRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba.index 18 | 19 | import org.apache.spark.sql.simba.partitioner.RangePartition 20 | import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation 21 | import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences} 22 | import org.apache.spark.sql.execution.SparkPlan 23 | import org.apache.spark.sql.types.NumericType 24 | import org.apache.spark.storage.StorageLevel 25 | 26 | /** 27 | * Created by dongx on 11/12/2016. 28 | */ 29 | private[simba] case class TreapIndexedRelation(output: Seq[Attribute], child: SparkPlan, 30 | table_name: Option[String], column_keys: List[Attribute], index_name: String) 31 | (var _indexedRDD: IndexedRDD = null, var range_bounds: Array[Double] = null) 32 | extends IndexedRelation with MultiInstanceRelation { 33 | 34 | require(column_keys.length == 1) 35 | require(column_keys.head.dataType.isInstanceOf[NumericType]) 36 | val numShufflePartitions = simbaSession.sessionState.simbaConf.indexPartitions 37 | 38 | if (_indexedRDD == null) { 39 | buildIndex() 40 | } 41 | 42 | private[simba] def buildIndex(): Unit = { 43 | val dataRDD = child.execute().map(row => { 44 | val eval_key = BindReferences.bindReference(column_keys.head, child.output).eval(row) 45 | .asInstanceOf[Double] 46 | (eval_key, row) 47 | }) 48 | 49 | val (partitionedRDD, tmp_bounds) = RangePartition.rowPartition(dataRDD, numShufflePartitions) 50 | range_bounds = tmp_bounds 51 | val indexed = partitionedRDD.mapPartitions(iter => { 52 | val data = iter.toArray 53 | val index = Treap(data) 54 | Array(IPartition(data.map(_._2), index)).iterator 55 | }).persist(StorageLevel.MEMORY_AND_DISK_SER) 56 | 57 | indexed.setName(table_name.map(n => s"$n $index_name").getOrElse(child.toString)) 58 | _indexedRDD = indexed 59 | } 60 | 61 | override def newInstance(): IndexedRelation = { 62 | TreapIndexedRelation(output.map(_.newInstance()), child, table_name, 63 | column_keys, index_name)(_indexedRDD) 64 | .asInstanceOf[this.type] 65 | } 66 | 67 | override def withOutput(new_output: Seq[Attribute]): IndexedRelation = { 68 | TreapIndexedRelation(new_output, child, table_name, 69 | column_keys, index_name)(_indexedRDD, range_bounds) 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/TreeMapIndexedRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba.index 18 | 19 | import org.apache.spark.sql.simba.partitioner.RangePartition 20 | import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation 21 | import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences} 22 | import org.apache.spark.sql.execution.SparkPlan 23 | import org.apache.spark.sql.types.NumericType 24 | import org.apache.spark.storage.StorageLevel 25 | 26 | /** 27 | * Created by dongx on 11/12/2016. 28 | */ 29 | private[simba] case class TreeMapIndexedRelation(output: Seq[Attribute], child: SparkPlan, 30 | table_name: Option[String], column_keys: List[Attribute], index_name: String) 31 | (var _indexedRDD: IndexedRDD = null, var range_bounds: Array[Double] = null) 32 | extends IndexedRelation with MultiInstanceRelation { 33 | 34 | require(column_keys.length == 1) 35 | require(column_keys.head.dataType.isInstanceOf[NumericType]) 36 | 37 | if (_indexedRDD == null) { 38 | buildIndex() 39 | } 40 | 41 | private[simba] def buildIndex(): Unit = { 42 | val numShufflePartitions = simbaSession.sessionState.simbaConf.indexPartitions 43 | 44 | val dataRDD = child.execute().map(row => { 45 | val eval_key = BindReferences.bindReference(column_keys.head, child.output).eval(row) 46 | .asInstanceOf[Double] 47 | (eval_key, row) 48 | }) 49 | 50 | val (partitionedRDD, tmp_bounds) = RangePartition.rowPartition(dataRDD, numShufflePartitions) 51 | range_bounds = tmp_bounds 52 | val indexed = partitionedRDD.mapPartitions(iter => { 53 | val data = iter.toArray 54 | val index = TreeMapIndex(data) 55 | Array(IPartition(data.map(_._2), index)).iterator 56 | }).persist(StorageLevel.MEMORY_AND_DISK_SER) 57 | 58 | indexed.setName(table_name.map(n => s"$n $index_name").getOrElse(child.toString)) 59 | _indexedRDD = indexed 60 | } 61 | 62 | override def newInstance(): IndexedRelation = { 63 | TreeMapIndexedRelation(output.map(_.newInstance()), child, table_name, 64 | column_keys, index_name)(_indexedRDD) 65 | .asInstanceOf[this.type] 66 | } 67 | 68 | override def withOutput(new_output: Seq[Attribute]): IndexedRelation = { 69 | TreeMapIndexedRelation(new_output, child, table_name, 70 | column_keys, index_name)(_indexedRDD, range_bounds) 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/examples/IndexExample.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.simba.examples 2 | 3 | import org.apache.spark.sql.simba.{Dataset, SimbaSession} 4 | import org.apache.spark.sql.simba.index.{RTreeType, TreapType} 5 | 6 | /** 7 | * Created by dongx on 3/7/2017. 8 | */ 9 | object IndexExample { 10 | case class PointData(x: Double, y: Double, z: Double, other: String) 11 | 12 | def main(args: Array[String]): Unit = { 13 | val simbaSession = SimbaSession 14 | .builder() 15 | .master("local[4]") 16 | .appName("IndexExample") 17 | .config("simba.index.partitions", "64") 18 | .getOrCreate() 19 | 20 | buildIndex(simbaSession) 21 | useIndex1(simbaSession) 22 | useIndex2(simbaSession) 23 | simbaSession.stop() 24 | } 25 | 26 | private def buildIndex(simba: SimbaSession): Unit = { 27 | import simba.implicits._ 28 | val datapoints = Seq(PointData(1.0, 1.0, 3.0, "1"), PointData(2.0, 2.0, 3.0, "2"), PointData(2.0, 2.0, 3.0, "3"), 29 | PointData(2.0, 2.0, 3.0, "4"),PointData(3.0, 3.0, 3.0, "5"),PointData(4.0, 4.0, 3.0, "6")).toDS 30 | 31 | datapoints.createOrReplaceTempView("a") 32 | 33 | simba.indexTable("a", RTreeType, "testqtree", Array("x", "y") ) 34 | 35 | simba.showIndex("a") 36 | } 37 | 38 | private def useIndex1(simba: SimbaSession): Unit = { 39 | import simba.implicits._ 40 | import simba.simbaImplicits._ 41 | val datapoints = Seq(PointData(1.0, 1.0, 3.0, "1"), PointData(2.0, 2.0, 3.0, "2"), PointData(2.0, 2.0, 3.0, "3"), 42 | PointData(2.0, 2.0, 3.0, "4"),PointData(3.0, 3.0, 3.0, "5"),PointData(4.0, 4.0, 3.0, "6")).toDF() 43 | 44 | datapoints.createOrReplaceTempView("b") 45 | 46 | simba.indexTable("b", RTreeType, "RtreeForData", Array("x", "y") ) 47 | 48 | simba.showIndex("b") 49 | 50 | val res = simba.sql("SELECT * FROM b") 51 | res.knn(Array("x", "y"),Array(1.0, 1.0),4).show(4) 52 | 53 | } 54 | 55 | private def useIndex2(simba: SimbaSession): Unit = { 56 | import simba.implicits._ 57 | val datapoints = Seq(PointData(1.0, 1.0, 3.0, "1"), PointData(2.0, 2.0, 3.0, "2"), PointData(2.0, 2.0, 3.0, "3"), 58 | PointData(2.0, 2.0, 3.0, "4"),PointData(3.0, 3.0, 3.0, "5"),PointData(4.0, 4.0, 3.0, "6")).toDF() 59 | 60 | datapoints.createOrReplaceTempView("b") 61 | 62 | simba.indexTable("b", RTreeType, "RtreeForData", Array("x", "y") ) 63 | 64 | simba.showIndex("b") 65 | 66 | simba.sql("SELECT * FROM b where b.x >1 and b.y<=2").show(5) 67 | 68 | } 69 | 70 | private def useIndex3(simba: SimbaSession): Unit = { 71 | import simba.implicits._ 72 | val datapoints = Seq(PointData(0.0, 1.0, 3.0, "1"), PointData(2.0, 2.0, 3.0, "2"), PointData(2.0, 2.0, 3.0, "3"), 73 | PointData(2.0, 2.0, 3.0, "4"),PointData(3.0, 3.0, 3.0, "5"),PointData(4.0, 4.0, 3.0, "6")).toDS() 74 | 75 | import simba.simbaImplicits._ 76 | 77 | datapoints.index(TreapType, "indexForOneTable", Array("x")) 78 | 79 | datapoints.range(Array("x"),Array(1.0),Array(2.0)).show(4) 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/Point.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.apache.spark.sql.simba.ShapeType 21 | import org.apache.spark.sql.types.SQLUserDefinedType 22 | 23 | /** 24 | * Created by dong on 1/15/16. 25 | * Multi-Dimensional Point 26 | */ 27 | @SQLUserDefinedType(udt = classOf[ShapeType]) 28 | case class Point(coord: Array[Double]) extends Shape { 29 | override val dimensions: Int = coord.length 30 | 31 | def this() = this(Array()) 32 | 33 | override def intersects(other: Shape): Boolean = { 34 | other match { 35 | case p: Point => p == this 36 | case mbr: MBR => mbr.contains(this) 37 | case cir: Circle => cir.contains(this) 38 | case poly: Polygon => poly.contains(this) 39 | case seg: LineSegment => seg.contains(this) 40 | } 41 | } 42 | 43 | override def minDist(other: Shape): Double = { 44 | other match { 45 | case p: Point => minDist(p) 46 | case mbr: MBR => mbr.minDist(this) 47 | case cir: Circle => cir.minDist(this) 48 | case poly: Polygon => poly.minDist(this) 49 | case seg: LineSegment => seg.minDist(this) 50 | } 51 | } 52 | 53 | def minDist(other: Point): Double = { 54 | require(coord.length == other.coord.length) 55 | var ans = 0.0 56 | for (i <- coord.indices) 57 | ans += (coord(i) - other.coord(i)) * (coord(i) - other.coord(i)) 58 | Math.sqrt(ans) 59 | } 60 | 61 | def ==(other: Point): Boolean = other match { 62 | case p: Point => 63 | if (p.coord.length != coord.length) false 64 | else { 65 | for (i <- coord.indices) 66 | if (coord(i) != p.coord(i)) return false 67 | true 68 | } 69 | case _ => false 70 | } 71 | 72 | def <=(other: Point): Boolean = { 73 | for (i <- coord.indices) 74 | if (coord(i) > other.coord(i)) return false 75 | true 76 | } 77 | 78 | def shift(d: Double): Point = Point(coord.map(x => x + d)) 79 | 80 | override def toString: String = { 81 | var s = "POINT(" 82 | s += coord(0).toString 83 | for (i <- 1 until coord.length) s += "," + coord(i) 84 | s + ")" 85 | } 86 | 87 | def getMBR: MBR = MBR(this, this) 88 | } 89 | 90 | object Point{ 91 | def apply(p: com.vividsolutions.jts.geom.Point): Point = { 92 | val coords = p.getCoordinate 93 | new Point(Array(coords.x, coords.y, coords.z)) 94 | } 95 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/util/ShapeUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.util 19 | 20 | import org.apache.spark.sql.simba.{ShapeSerializer, ShapeType} 21 | import org.apache.spark.sql.simba.expression.PointWrapper 22 | import org.apache.spark.sql.simba.spatial.{Point, Shape} 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences, Expression, UnsafeArrayData} 25 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 26 | import org.apache.spark.sql.execution.SparkPlan 27 | 28 | /** 29 | * Created by dongx on 11/12/2016. 30 | */ 31 | object ShapeUtils { 32 | def getPointFromRow(row: InternalRow, columns: List[Attribute], plan: SparkPlan, 33 | isPoint: Boolean): Point = { 34 | if (isPoint) { 35 | ShapeSerializer.deserialize(BindReferences.bindReference(columns.head, plan.output) 36 | .eval(row).asInstanceOf[UnsafeArrayData].toByteArray).asInstanceOf[Point] 37 | } else { 38 | Point(columns.toArray.map(BindReferences.bindReference(_, plan.output).eval(row) 39 | .asInstanceOf[Number].doubleValue())) 40 | } 41 | } 42 | def getPointFromRow(row: InternalRow, columns: List[Attribute], plan: LogicalPlan, 43 | isPoint: Boolean): Point = { 44 | if (isPoint) { 45 | ShapeSerializer.deserialize(BindReferences.bindReference(columns.head, plan.output) 46 | .eval(row).asInstanceOf[UnsafeArrayData].toByteArray).asInstanceOf[Point] 47 | } else { 48 | Point(columns.toArray.map(BindReferences.bindReference(_, plan.output).eval(row) 49 | .asInstanceOf[Number].doubleValue())) 50 | } 51 | } 52 | 53 | def getShape(expression: Expression, input: InternalRow): Shape = { 54 | if (!expression.isInstanceOf[PointWrapper] && expression.dataType.isInstanceOf[ShapeType]) { 55 | ShapeSerializer.deserialize(expression.eval(input).asInstanceOf[UnsafeArrayData].toByteArray) 56 | } else if (expression.isInstanceOf[PointWrapper]) { 57 | expression.eval(input).asInstanceOf[Shape] 58 | } else throw new UnsupportedOperationException("Query shape should be of ShapeType") 59 | } 60 | 61 | def getShape(expression: Expression, schema: Seq[Attribute], input: InternalRow): Shape = { 62 | if (!expression.isInstanceOf[PointWrapper] && expression.dataType.isInstanceOf[ShapeType]) { 63 | ShapeSerializer.deserialize(BindReferences.bindReference(expression, schema) 64 | .eval(input).asInstanceOf[UnsafeArrayData].toByteArray) 65 | } else if (expression.isInstanceOf[PointWrapper]) { 66 | BindReferences.bindReference(expression, schema).eval(input).asInstanceOf[Shape] 67 | } else throw new UnsupportedOperationException("Query shape should be of ShapeType") 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/BDJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.partitioner.MapDPartition 22 | import org.apache.spark.sql.simba.spatial.Point 23 | import org.apache.spark.sql.simba.util.{NumberUtil, ShapeUtils} 24 | import org.apache.spark.rdd.RDD 25 | import org.apache.spark.sql.catalyst.InternalRow 26 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 27 | import org.apache.spark.sql.execution.SparkPlan 28 | 29 | import scala.collection.mutable 30 | import scala.util.Random 31 | 32 | /** 33 | * Created by dongx on 11/11/16. 34 | */ 35 | case class BDJSpark(left_key: Expression, right_key: Expression, l: Literal, 36 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 37 | override def output: Seq[Attribute] = left.output ++ right.output 38 | 39 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 40 | final val r = NumberUtil.literalToDouble(l) 41 | 42 | override protected def doExecute(): RDD[InternalRow] = { 43 | val tot_rdd = left.execute().map((0, _)).union(right.execute().map((1, _))) 44 | 45 | val tot_dup_rdd = tot_rdd.flatMap {x => 46 | val rand_no = new Random().nextInt(num_partitions) 47 | var ans = mutable.ListBuffer[(Int, (Int, InternalRow))]() 48 | if (x._1 == 0) { 49 | val base = rand_no * num_partitions 50 | for (i <- 0 until num_partitions) 51 | ans += ((base + i, x)) 52 | } else { 53 | for (i <- 0 until num_partitions) 54 | ans += ((i * num_partitions + rand_no, x)) 55 | } 56 | ans 57 | } 58 | 59 | val tot_dup_partitioned = MapDPartition(tot_dup_rdd, num_partitions * num_partitions) 60 | 61 | tot_dup_partitioned.mapPartitions {iter => 62 | var left_data = mutable.ListBuffer[(Point, InternalRow)]() 63 | var right_data = mutable.ListBuffer[(Point, InternalRow)]() 64 | while (iter.hasNext) { 65 | val data = iter.next() 66 | if (data._2._1 == 0) { 67 | val tmp_point = ShapeUtils.getShape(left_key, left.output, data._2._2).asInstanceOf[Point] 68 | left_data += ((tmp_point, data._2._2)) 69 | } else { 70 | val tmp_point = ShapeUtils.getShape(right_key, right.output, data._2._2).asInstanceOf[Point] 71 | right_data += ((tmp_point, data._2._2)) 72 | } 73 | } 74 | 75 | val joined_ans = mutable.ListBuffer[InternalRow]() 76 | 77 | left_data.foreach {left => 78 | right_data.foreach {right => 79 | if (left._1.minDist(right._1) <= r) { 80 | joined_ans += new JoinedRow(left._2, right._2) 81 | } 82 | } 83 | } 84 | 85 | joined_ans.iterator 86 | } 87 | } 88 | 89 | override def children: Seq[SparkPlan] = Seq(left, right) 90 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/SimbaOptimizer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba 19 | 20 | import org.apache.spark.sql.ExperimentalMethods 21 | import org.apache.spark.sql.catalyst.catalog.SessionCatalog 22 | import org.apache.spark.sql.catalyst.expressions.{And, Expression, PredicateHelper} 23 | import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} 24 | import org.apache.spark.sql.catalyst.rules.Rule 25 | import org.apache.spark.sql.execution.SparkOptimizer 26 | import org.apache.spark.sql.internal.SQLConf 27 | import org.apache.spark.sql.simba.plans.SpatialJoin 28 | 29 | /** 30 | * Created by dongx on 3/7/17. 31 | */ 32 | class SimbaOptimizer(catalog: SessionCatalog, 33 | conf: SQLConf, 34 | experimentalMethods: ExperimentalMethods) 35 | extends SparkOptimizer(catalog, conf, experimentalMethods) { 36 | override def batches: Seq[Batch] = super.batches :+ 37 | Batch("SpatialJoinPushDown", FixedPoint(100), PushPredicateThroughSpatialJoin) 38 | } 39 | 40 | object PushPredicateThroughSpatialJoin extends Rule[LogicalPlan] with PredicateHelper { 41 | private def split(condition: Seq[Expression], left: LogicalPlan, right: LogicalPlan) = { 42 | val (leftEvaluateCondition, rest) = 43 | condition.partition(_.references subsetOf left.outputSet) 44 | val (rightEvaluateCondition, commonCondition) = 45 | rest.partition(_.references subsetOf right.outputSet) 46 | 47 | (leftEvaluateCondition, rightEvaluateCondition, commonCondition) 48 | } 49 | 50 | def apply(plan: LogicalPlan): LogicalPlan = plan transform { 51 | // push the where condition down into join filter 52 | case f @ Filter(filterCondition, SpatialJoin(left, right, joinType, joinCondition)) => 53 | val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = 54 | split(splitConjunctivePredicates(filterCondition), left, right) 55 | 56 | val newLeft = leftFilterConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) 57 | val newRight = rightFilterConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) 58 | val newJoinCond = (commonFilterCondition ++ joinCondition).reduceLeftOption(And) 59 | SpatialJoin(newLeft, newRight, joinType, newJoinCond) 60 | 61 | // push down the join filter into sub query scanning if applicable 62 | case f @ SpatialJoin(left, right, joinType, joinCondition) => 63 | val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = 64 | split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) 65 | 66 | val newLeft = leftJoinConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) 67 | val newRight = rightJoinConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) 68 | val newJoinCond = commonJoinCondition.reduceLeftOption(And) 69 | 70 | SpatialJoin(newLeft, newRight, joinType, newJoinCond) 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Simba: Spatial In-Memory Big data Analytics 2 | =========================================== 3 | **Simba is now shipped as a standalone package outside Spark. Current version works with Spark 2.1.x. If you find any issues, please make a ticket in the issue tracking system.** 4 | 5 | Simba is a distributed in-memory spatial analytics engine based on Apache Spark. It extends the Spark SQL engine across the system stack to support rich spatial queries and analytics through both SQL and the DataFrame API. Besides, Simba introduces native indexing support over RDDs in order to develop efficient spatial operators. It also extends Spark SQL's query optimizer with spatial-aware and cost-based optimizations to make the best use of existing indexes and statistics. 6 | 7 | Simba is open sourced under Apache License 2.0. Currently, it is developed based on Spark 1.6.0. For recent updates and further information, please refer to [Simba's homepage](http://www.cs.utah.edu/~dongx/simba). 8 | 9 | Features 10 | -------------- 11 | + Expressive **SQL and DataFrame query interface** fully *compatible with original Spark SQL operators*. (SQL mode is currently not supported in the standalone version.) 12 | + Native distributed **indexing** support over RDDs. 13 | + Efficient **spatial operators**: *high-throughput* & *low-latency*. 14 | - Box range query: `IN RANGE` 15 | - Circle range query: `IN CIRCLERANGE` 16 | - *k* nearest neighbor query: `IN KNN` 17 | - Distance join: `DISTANCE JOIN` 18 | - kNN join: `KNN JOIN` 19 | + Modified Zeppelin: **interactive visualization** for Simba. 20 | + Spatial-aware **optimizations**: *logical* & *cost-based*. 21 | + Native thread-pool for multi-threading. 22 | + **Geometric objects** support (developing) 23 | + **Spatio-Temporal** and **spatio-textual** data analysis (developing) 24 | 25 | **Notes:** *We are still cleaning source codes for some of our features, which will be released to the master and develop branch later.* 26 | 27 | Developer Notes 28 | --------------- 29 | 1. Fork this repo (or create your own branch if you are a member of Simba's main development team) to start your development, **DO NOT** push your draft version to the master branch 30 | 2. You can build your own application in `org.apache.spark.examples` package for testing or debugging. 31 | 3. If you want to merge your feature branch to the main develop branch, please create a pull request from your local branch to develop branch (**not the master branch**). 32 | 33 | Branch Information 34 | ------------------ 35 | `standalone` branches are opened for maintaining Simba standalone package, which aims at building Simba packages standing outside Spark SQL core. Currently, `master` branch and `develop` branch are built on top of Spark 2.1.x. 36 | 37 | The `master` branch provides the latest stable version, while the `develop` branch is the main development branch where new features will be merged before ready to release. For legacy reasons, we also keep branches which archives old versions of Simba, which is developed based on former Spark versions, in the branches named `simba-spark-x.x`. Note that we will only integrate latest features into `master` and `develop` branches. Please make sure you checkout the correct branch before start using it. 38 | 39 | Contributors 40 | ------------ 41 | - Dong Xie: dongx [at] cs [dot] utah [dot] edu 42 | - Gefei Li: oizz01 [at] sjtu [dot] edu [dot] cn 43 | - Liang Zhou: nichozl [at] sjtu [dot] edu [dot] cn 44 | - Zhongpu Chen: chenzhongpu [at] sjtu [dot] edu [dot] cn 45 | - Feifei Li: lifeifei [at] cs [dot] utah [dot] edu 46 | - Bin Yao: yaobin [at] cs [dot] sjtu [dot] edu [dot] cn 47 | - Minyi Guo: guo-my [at] cs [dot] sjtu [dot] edu [dot] cn 48 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/QuadTreeIndexedRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License 15 | */ 16 | package org.apache.spark.sql.simba.index 17 | 18 | import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation 19 | import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences} 20 | import org.apache.spark.sql.execution.SparkPlan 21 | import org.apache.spark.sql.types.{DoubleType, IntegerType} 22 | import org.apache.spark.storage.StorageLevel 23 | 24 | import org.apache.spark.sql.simba.partitioner.QuadTreePartitioner 25 | import org.apache.spark.sql.simba.spatial.Point 26 | 27 | /** 28 | * Created by gefei on 16-7-30. 29 | */ 30 | private[simba] case class QuadTreeIndexedRelation(output: Seq[Attribute], child: SparkPlan, table_name: Option[String], 31 | column_keys: List[Attribute], index_name: String)(var _indexedRDD: IndexedRDD = null, var global_index: QuadTree = null) 32 | extends IndexedRelation with MultiInstanceRelation { 33 | private def checkKeys: Boolean = { 34 | for (i <- column_keys.indices) 35 | if (!(column_keys(i).dataType.isInstanceOf[DoubleType] || 36 | column_keys(i).dataType.isInstanceOf[IntegerType])) { 37 | return false 38 | } 39 | true 40 | } 41 | require(checkKeys) 42 | 43 | if (_indexedRDD == null) { 44 | buildIndex() 45 | } 46 | 47 | private[simba] def buildIndex(): Unit = { 48 | val numShufflePartitions = simbaSession.sessionState.simbaConf.indexPartitions 49 | val sampleRate = simbaSession.sessionState.simbaConf.sampleRate 50 | val tranferThreshold = simbaSession.sessionState.simbaConf.transferThreshold 51 | 52 | val dataRDD = child.execute().map(row => { 53 | val now = column_keys.map(x => 54 | BindReferences.bindReference(x, child.output).eval(row).asInstanceOf[Number].doubleValue() 55 | ).toArray 56 | (new Point(now), row) 57 | }) 58 | 59 | val dimension = column_keys.length 60 | val (partitionedRDD, _, global_qtree) = QuadTreePartitioner(dataRDD, dimension, 61 | numShufflePartitions, sampleRate, tranferThreshold) 62 | 63 | val indexed = partitionedRDD.mapPartitions { iter => 64 | val data = iter.toArray 65 | val index: QuadTree = 66 | if (data.length > 0) QuadTree(data.map(_._1).zipWithIndex) 67 | else null 68 | Array(IPartition(data.map(_._2), index)).iterator 69 | }.persist(StorageLevel.MEMORY_AND_DISK_SER) 70 | 71 | indexed.setName(table_name.map(name => s"$name $index_name").getOrElse(child.toString)) 72 | _indexedRDD = indexed 73 | global_index = global_qtree 74 | } 75 | 76 | override def newInstance(): IndexedRelation = { 77 | new QuadTreeIndexedRelation(output.map(_.newInstance()), child, table_name, 78 | column_keys, index_name)(_indexedRDD) 79 | .asInstanceOf[this.type] 80 | } 81 | 82 | override def withOutput(new_output: Seq[Attribute]): IndexedRelation = { 83 | new QuadTreeIndexedRelation(new_output, child, table_name, 84 | column_keys, index_name)(_indexedRDD, global_index) 85 | } 86 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/RDJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.index.RTree 22 | import org.apache.spark.sql.simba.partitioner.{MapDPartition, STRPartition} 23 | import org.apache.spark.sql.simba.spatial.Point 24 | import org.apache.spark.sql.simba.util.{NumberUtil, ShapeUtils} 25 | import org.apache.spark.rdd.RDD 26 | import org.apache.spark.sql.catalyst.InternalRow 27 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 28 | import org.apache.spark.sql.execution.SparkPlan 29 | 30 | import scala.collection.mutable 31 | 32 | /** 33 | * Created by dong on 1/20/16. 34 | * Distance Join based on Two-Level R-Tree Structure 35 | */ 36 | case class RDJSpark(left_key: Expression, right_key: Expression, l: Literal, 37 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 38 | override def output: Seq[Attribute] = left.output ++ right.output 39 | 40 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 41 | final val sample_rate = simbaSessionState.simbaConf.sampleRate 42 | final val max_entries_per_node = simbaSessionState.simbaConf.maxEntriesPerNode 43 | final val transfer_threshold = simbaSessionState.simbaConf.transferThreshold 44 | final val r = NumberUtil.literalToDouble(l) 45 | 46 | override protected def doExecute(): RDD[InternalRow] = { 47 | val left_rdd = left.execute().map(row => 48 | (ShapeUtils.getShape(left_key, left.output, row).asInstanceOf[Point], row) 49 | ) 50 | 51 | val right_rdd = right.execute().map(row => 52 | (ShapeUtils.getShape(right_key, right.output, row).asInstanceOf[Point], row) 53 | ) 54 | 55 | val dimension = right_rdd.first()._1.coord.length 56 | 57 | val (left_partitioned, left_mbr_bound) = 58 | STRPartition(left_rdd, dimension, num_partitions, sample_rate, 59 | transfer_threshold, max_entries_per_node) 60 | 61 | val left_part_size = left_partitioned.mapPartitions { 62 | iter => Array(iter.length).iterator 63 | }.collect() 64 | 65 | val left_rt = RTree(left_mbr_bound.zip(left_part_size).map(x => (x._1._1, x._1._2, x._2)), 66 | max_entries_per_node) 67 | val bc_rt = sparkContext.broadcast(left_rt) 68 | 69 | val right_dup = right_rdd.flatMap {x => 70 | bc_rt.value.circleRange(x._1, r).map(now => (now._2, x)) 71 | } 72 | 73 | val right_dup_partitioned = MapDPartition(right_dup, left_mbr_bound.length) 74 | 75 | left_partitioned.zipPartitions(right_dup_partitioned) {(leftIter, rightIter) => 76 | val ans = mutable.ListBuffer[InternalRow]() 77 | val right_data = rightIter.map(_._2).toArray 78 | if (right_data.length > 0) { 79 | val right_index = RTree(right_data.map(_._1).zipWithIndex, max_entries_per_node) 80 | leftIter.foreach {now => 81 | ans ++= right_index.circleRange(now._1, r) 82 | .map(x => new JoinedRow(now._2, right_data(x._2)._2)) 83 | } 84 | } 85 | ans.iterator 86 | } 87 | } 88 | 89 | override def children: Seq[SparkPlan] = Seq(left, right) 90 | } 91 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/BDJSparkR.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.index.RTree 22 | import org.apache.spark.sql.simba.partitioner.MapDPartition 23 | import org.apache.spark.sql.simba.spatial.Point 24 | import org.apache.spark.sql.simba.util.{NumberUtil, ShapeUtils} 25 | import org.apache.spark.rdd.RDD 26 | import org.apache.spark.sql.catalyst.InternalRow 27 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 28 | import org.apache.spark.sql.execution.SparkPlan 29 | 30 | import scala.collection.mutable 31 | import scala.util.Random 32 | 33 | /** 34 | * Created by dong on 1/20/16. 35 | * Distance Join based on Block Nested Loop + Local R-Tree 36 | */ 37 | case class BDJSparkR(left_key: Expression, right_key: Expression, l: Literal, 38 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 39 | override def output: Seq[Attribute] = left.output ++ right.output 40 | 41 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 42 | final val r = NumberUtil.literalToDouble(l) 43 | final val max_entries_per_node = simbaSessionState.simbaConf.maxEntriesPerNode 44 | 45 | override protected def doExecute(): RDD[InternalRow] = { 46 | val tot_rdd = left.execute().map((0, _)).union(right.execute().map((1, _))) 47 | 48 | val tot_dup_rdd = tot_rdd.flatMap {x => 49 | val rand_no = new Random().nextInt(num_partitions) 50 | var ans = mutable.ListBuffer[(Int, (Int, InternalRow))]() 51 | if (x._1 == 0) { 52 | val base = rand_no * num_partitions 53 | for (i <- 0 until num_partitions) 54 | ans += ((base + i, x)) 55 | } else { 56 | for (i <- 0 until num_partitions) 57 | ans += ((i * num_partitions + rand_no, x)) 58 | } 59 | ans 60 | } 61 | 62 | val tot_dup_partitioned = MapDPartition(tot_dup_rdd, num_partitions * num_partitions) 63 | 64 | tot_dup_partitioned.mapPartitions {iter => 65 | var left_data = mutable.ListBuffer[(Point, InternalRow)]() 66 | var right_data = mutable.ListBuffer[(Point, InternalRow)]() 67 | while (iter.hasNext) { 68 | val data = iter.next() 69 | if (data._2._1 == 0) { 70 | val tmp_point = ShapeUtils.getShape(left_key, left.output, data._2._2).asInstanceOf[Point] 71 | left_data += ((tmp_point, data._2._2)) 72 | } else { 73 | val tmp_point = ShapeUtils.getShape(right_key, right.output, data._2._2).asInstanceOf[Point] 74 | right_data += ((tmp_point, data._2._2)) 75 | } 76 | } 77 | 78 | val joined_ans = mutable.ListBuffer[InternalRow]() 79 | 80 | if (right_data.nonEmpty) { 81 | val right_rtree = RTree(right_data.map(_._1).zipWithIndex.toArray, max_entries_per_node) 82 | left_data.foreach(left => right_rtree.circleRange(left._1, r) 83 | .foreach(x => joined_ans += new JoinedRow(left._2, right_data(x._2)._2))) 84 | } 85 | 86 | joined_ans.iterator 87 | } 88 | } 89 | 90 | override def children: Seq[SparkPlan] = Seq(left, right) 91 | } 92 | 93 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/util/PredicateUtil.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.util 19 | 20 | import org.apache.spark.sql.catalyst.expressions.{Expression, And, Or} 21 | 22 | /** 23 | * Created by gefei on 2016/11/15. 24 | */ 25 | object PredicateUtil { 26 | def toDNF(condition: Expression): Expression = { 27 | condition match { 28 | case Or(left, right) => 29 | Or(toDNF(left), toDNF(right)) 30 | case And(left, right) => 31 | var ans: Expression = null 32 | val tmp_left = toDNF(left) 33 | val tmp_right = toDNF(right) 34 | tmp_left match { 35 | case Or(l, r) => 36 | ans = Or(And(l, tmp_right), And(r, tmp_right)) 37 | case _ => 38 | } 39 | tmp_right match { 40 | case Or(l, r) => 41 | if (ans == null) ans = Or(And(tmp_left, l), And(tmp_left, r)) 42 | case _ => 43 | } 44 | if (ans == null) And(tmp_left, tmp_right) 45 | else toDNF(ans) 46 | case exp => exp 47 | } 48 | } 49 | 50 | def toCNF(condition: Expression): Expression = { 51 | condition match { 52 | case And(left, right) => 53 | And(toCNF(left), toCNF(right)) 54 | case Or(left, right) => 55 | var ans: Expression = null 56 | val tmp_left = toCNF(left) 57 | val tmp_right = toCNF(right) 58 | tmp_left match { 59 | case And(l, r) => 60 | ans = And(Or(l, tmp_right), Or(r, tmp_right)) 61 | case _ => 62 | } 63 | tmp_right match { 64 | case And(l, r) => 65 | if (ans == null) ans = And(Or(tmp_left, l), Or(tmp_left, r)) 66 | case _ => 67 | } 68 | if (ans == null) Or(tmp_left, tmp_right) 69 | else toCNF(ans) 70 | case exp => exp 71 | } 72 | } 73 | def dnfExtract(expression: Expression): Seq[Expression] = { 74 | expression match { 75 | case Or(left, right) => 76 | dnfExtract(left) ++ dnfExtract(right) 77 | case And(left @ And(l2, r2), right) => 78 | dnfExtract(And(l2, And(r2, right))) 79 | case other => 80 | other :: Nil 81 | } 82 | } 83 | 84 | def cnfExtract(expression: Expression): Seq[Expression] = { 85 | expression match { 86 | case And(left, right) => 87 | cnfExtract(left) ++ cnfExtract(right) 88 | case Or(left @ Or(l2, r2), right) => 89 | cnfExtract(Or(l2, Or(r2, right))) 90 | case other => 91 | other :: Nil 92 | } 93 | } 94 | 95 | def splitDNFPredicates(condition: Expression) = dnfExtract(toDNF(condition)) 96 | 97 | def splitCNFPredicates(condition: Expression) = cnfExtract(toCNF(condition)) 98 | 99 | 100 | def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { 101 | condition match { 102 | case And(cond1, cond2) => 103 | splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) 104 | case other => other :: Nil 105 | } 106 | } 107 | 108 | def splitDisjunctivePredicates(condition: Expression): Seq[Expression] = { 109 | condition match { 110 | case Or(cond1, cond2) => 111 | splitDisjunctivePredicates(cond1) ++ splitDisjunctivePredicates(cond2) 112 | case other => other :: Nil 113 | } 114 | } 115 | 116 | } 117 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/index/IntervalSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | import org.scalatest.FunSuite 21 | import org.apache.spark.sql.catalyst.expressions._ 22 | 23 | /** 24 | * Created by Zhihao Bai on 16-8-3. 25 | */ 26 | class IntervalSuite extends FunSuite{ 27 | def equal(A: Interval, B: Interval): Boolean = { 28 | A.min == B.min && A.max == B.max 29 | } 30 | 31 | test("Interval: isNull"){ 32 | val i = new Interval((0.0, false), (0.0, true)) 33 | assert(i.isNull) 34 | } 35 | 36 | test("Interval: intersect"){ 37 | val I = new Interval((-1.0, true), (1.0, true)) 38 | val J = new Interval((0.0, false), (2.0, false)) 39 | val K1 = new Interval((1.0, true), (3.0, false)) 40 | val K2 = new Interval((1.0, false), (3.0, false)) 41 | 42 | assert(equal(I.intersect(J), new Interval((0.0, false), (1.0, true)))) 43 | assert(equal(I.intersect(K1), new Interval((1.0, true), (1.0, true)))) 44 | assert(equal(I.intersect(K2), new Interval((1.0, false), (1.0, true)))) 45 | } 46 | 47 | test("Interval: getLeafInterval"){ 48 | val left = new Alias(Literal.apply(0), "zero")() 49 | val right = Literal.apply(10) 50 | 51 | val eq = new EqualTo(left, right) 52 | assert(equal(Interval.getLeafInterval(eq)._1, 53 | new Interval((10.0, true), (10.0, true)))) 54 | 55 | val lt = new LessThan(left, right) 56 | assert(equal(Interval.getLeafInterval(lt)._1, 57 | new Interval((Double.MinValue, false), (10.0, false)))) 58 | 59 | val leq = new LessThanOrEqual(left, right) 60 | assert(equal(Interval.getLeafInterval(leq)._1, 61 | new Interval((Double.MinValue, false), (10.0, true)))) 62 | 63 | val gt = new GreaterThan(left, right) 64 | assert(equal(Interval.getLeafInterval(gt)._1, 65 | new Interval((10.0, false), (Double.MaxValue, false)))) 66 | 67 | val geq = new GreaterThanOrEqual(left, right) 68 | assert(equal(Interval.getLeafInterval(geq)._1, 69 | new Interval((10.0, true), (Double.MaxValue, false)))) 70 | } 71 | 72 | test("Interval: conditionToInterval"){ 73 | val zero = new Alias(Literal.apply(0), "zero")() 74 | val ten = new Alias(Literal.apply(10), "ten")() 75 | 76 | val lt = new LessThan(zero, Literal.apply(10)) 77 | val gt = new GreaterThan(ten, Literal.apply(0)) 78 | 79 | val and = new And(lt, gt) 80 | val column = ten.toAttribute :: zero.toAttribute :: Nil 81 | 82 | val r = Interval.conditionToInterval(and, column, 2)._1 83 | assert(equal(r(0), new Interval((0.0, false), (Double.MaxValue,false)))) 84 | assert(equal(r(1), new Interval((Double.MinValue, false), (10.0,false)))) 85 | } 86 | 87 | test("Interval: getBoundNumberForInterval"){ 88 | def equalArray(x: Seq[Int], y: Seq[Int]): Boolean = { 89 | if(x.length != y.length) 90 | false 91 | else{ 92 | var eq = true 93 | for(i <- x.indices) 94 | eq = eq && (x(i) == y(i)) 95 | eq 96 | } 97 | } 98 | 99 | val I = new Interval(2.5, 4.5) 100 | val J = new Interval(5.5, 6.5) 101 | val A = Array(1.0, 2.0, 3.0, 4.0, 5.0) 102 | 103 | val x = Interval.getBoundNumberForInterval(I, A).sorted 104 | assert(equalArray(x, Array(2, 3, 4, 5))) 105 | 106 | val y = Interval.getBoundNumberForInterval(J, A).sorted 107 | assert(equalArray(y, Array(5))) 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/BKJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.partitioner.MapDPartition 22 | import org.apache.spark.sql.simba.spatial.Point 23 | import org.apache.spark.sql.simba.util.ShapeUtils 24 | import org.apache.spark.rdd.RDD 25 | import org.apache.spark.sql.catalyst.InternalRow 26 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 27 | import org.apache.spark.sql.execution.SparkPlan 28 | import org.apache.spark.util.BoundedPriorityQueue 29 | 30 | import scala.collection.mutable 31 | import scala.util.Random 32 | 33 | /** 34 | * Created by dong on 1/20/16. 35 | * KNN Join based on Block Nested Loop Approach 36 | */ 37 | case class BKJSpark(left_key: Expression, right_key: Expression, l: Literal, 38 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 39 | override def output: Seq[Attribute] = left.output ++ right.output 40 | 41 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 42 | final val k = l.value.asInstanceOf[Number].intValue() 43 | 44 | private class DisOrdering extends Ordering[(InternalRow, Double)] { 45 | override def compare(x : (InternalRow, Double), y: (InternalRow, Double)): Int = 46 | -x._2.compare(y._2) 47 | } 48 | 49 | override protected def doExecute(): RDD[InternalRow] = { 50 | val tot_rdd = left.execute().map((0, _)).union(right.execute().map((1, _))) 51 | 52 | val tot_dup_rdd = tot_rdd.flatMap {x => 53 | val rand_no = new Random().nextInt(num_partitions) 54 | val ans = mutable.ListBuffer[(Int, (Int, InternalRow))]() 55 | if (x._1 == 0) { 56 | val base = rand_no * num_partitions 57 | for (i <- 0 until num_partitions) 58 | ans += ((base + i, x)) 59 | } else { 60 | for (i <- 0 until num_partitions) 61 | ans += ((i * num_partitions + rand_no, x)) 62 | } 63 | ans 64 | } 65 | 66 | val tot_dup_partitioned = MapDPartition(tot_dup_rdd, num_partitions * num_partitions) 67 | 68 | tot_dup_partitioned.mapPartitions {iter => 69 | var left_data = mutable.ListBuffer[(Point, InternalRow)]() 70 | var right_data = mutable.ListBuffer[(Point, InternalRow)]() 71 | while (iter.hasNext) { 72 | val data = iter.next() 73 | if (data._2._1 == 0) { 74 | val tmp_point = ShapeUtils.getShape(left_key, left.output, data._2._2).asInstanceOf[Point] 75 | left_data += ((tmp_point, data._2._2)) 76 | } else { 77 | val tmp_point = ShapeUtils.getShape(right_key, right.output, data._2._2).asInstanceOf[Point] 78 | right_data += ((tmp_point, data._2._2)) 79 | } 80 | } 81 | 82 | val joined_ans = mutable.ListBuffer[(InternalRow, Array[(InternalRow, Double)])]() 83 | 84 | left_data.foreach(left => { 85 | var pq = new BoundedPriorityQueue[(InternalRow, Double)](k)(new DisOrdering) 86 | right_data.foreach(right => pq += ((right._2, right._1.minDist(left._1)))) 87 | joined_ans += ((left._2, pq.toArray)) 88 | }) 89 | joined_ans.iterator 90 | }.reduceByKey((left, right) => (left ++ right).sortWith(_._2 < _._2).take(k), num_partitions) 91 | .flatMap { 92 | now => now._2.map(x => new JoinedRow(now._1, x._1)) 93 | } 94 | } 95 | 96 | override def children: Seq[SparkPlan] = Seq(left, right) 97 | } 98 | 99 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/FilterExec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba.execution 18 | 19 | import org.apache.spark.sql.simba.expression._ 20 | import org.apache.spark.sql.simba.spatial.Point 21 | import org.apache.spark.sql.simba.util.ShapeUtils 22 | import org.apache.spark.rdd.RDD 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal, PredicateHelper} 25 | import org.apache.spark.sql.catalyst.expressions.{SortOrder, And => SQLAnd, Not => SQLNot, Or => SQLOr} 26 | import org.apache.spark.sql.catalyst.plans.physical.Partitioning 27 | import org.apache.spark.sql.execution.SparkPlan 28 | 29 | /** 30 | * Created by dongx on 11/13/2016. 31 | */ 32 | case class FilterExec(condition: Expression, child: SparkPlan) extends SimbaPlan with PredicateHelper { 33 | override def output: Seq[Attribute] = child.output 34 | 35 | private class DistanceOrdering(point: Expression, target: Point) extends Ordering[InternalRow] { 36 | override def compare(x: InternalRow, y: InternalRow): Int = { 37 | val shape_x = ShapeUtils.getShape(point, child.output, x) 38 | val shape_y = ShapeUtils.getShape(point, child.output, y) 39 | val dis_x = target.minDist(shape_x) 40 | val dis_y = target.minDist(shape_y) 41 | dis_x.compare(dis_y) 42 | } 43 | } 44 | 45 | // TODO change target partition from 1 to some good value 46 | // Note that target here must be an point literal in WHERE clause, 47 | // hence we can consider it as Point safely 48 | def knn(rdd: RDD[InternalRow], point: Expression, target: Point, k: Int): RDD[InternalRow] = 49 | sparkContext.parallelize(rdd.map(_.copy()).takeOrdered(k)(new DistanceOrdering(point, target)), 1) 50 | 51 | def applyCondition(rdd: RDD[InternalRow], condition: Expression): RDD[InternalRow] = { 52 | condition match { 53 | case InKNN(point, target, k) => 54 | val _target = target.asInstanceOf[Literal].value.asInstanceOf[Point] 55 | knn(rdd, point, _target, k.value.asInstanceOf[Number].intValue()) 56 | case now@And(left, right) => 57 | if (!now.hasKNN) rdd.mapPartitions{ iter => iter.filter(newPredicate(condition, child.output).eval(_))} 58 | else applyCondition(rdd, left).map(_.copy()).intersection(applyCondition(rdd, right).map(_.copy())) 59 | case now@Or(left, right) => 60 | if (!now.hasKNN) rdd.mapPartitions{ iter => iter.filter(newPredicate(condition, child.output).eval(_))} 61 | else applyCondition(rdd, left).map(_.copy()).union(applyCondition(rdd, right).map(_.copy())).distinct() 62 | case now@Not(c) => 63 | if (!now.hasKNN) rdd.mapPartitions{ iter => iter.filter(newPredicate(condition, child.output).eval(_))} 64 | else rdd.map(_.copy()).subtract(applyCondition(rdd, c).map(_.copy())) 65 | case _ => 66 | rdd.mapPartitions(iter => iter.filter(newPredicate(condition, child.output).eval(_))) 67 | } 68 | } 69 | 70 | protected def doExecute(): RDD[InternalRow] = { 71 | val root_rdd = child.execute() 72 | condition transformUp { 73 | case SQLAnd(left, right) => And(left, right) 74 | case SQLOr(left, right)=> Or(left, right) 75 | case SQLNot(c) => Not(c) 76 | } 77 | applyCondition(root_rdd, condition) 78 | } 79 | 80 | override def outputOrdering: Seq[SortOrder] = child.outputOrdering 81 | 82 | override def children: Seq[SparkPlan] = child :: Nil 83 | override def outputPartitioning: Partitioning = child.outputPartitioning 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/RTreeIndexedRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.simba.index 18 | 19 | import org.apache.spark.sql.simba.ShapeType 20 | import org.apache.spark.sql.simba.partitioner.STRPartition 21 | import org.apache.spark.sql.simba.util.ShapeUtils 22 | import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation 23 | import org.apache.spark.sql.catalyst.expressions.Attribute 24 | import org.apache.spark.sql.execution.SparkPlan 25 | import org.apache.spark.sql.types.NumericType 26 | import org.apache.spark.storage.StorageLevel 27 | 28 | /** 29 | * Created by dongx on 11/12/2016. 30 | */ 31 | private[simba] case class RTreeIndexedRelation(output: Seq[Attribute], child: SparkPlan, table_name: Option[String], 32 | column_keys: List[Attribute], index_name: String)(var _indexedRDD: IndexedRDD = null, var global_rtree: RTree = null) 33 | extends IndexedRelation with MultiInstanceRelation { 34 | 35 | var isPoint = false 36 | 37 | private def checkKeys: Boolean = { 38 | if (column_keys.length > 1) { 39 | for (i <- column_keys.indices) 40 | if (!column_keys(i).dataType.isInstanceOf[NumericType]) { 41 | return false 42 | } 43 | true 44 | } else { // length = 1; we do not support one dimension R-tree 45 | column_keys.head.dataType match { 46 | case t: ShapeType => 47 | isPoint = true 48 | true 49 | case _ => false 50 | } 51 | } 52 | } 53 | require(checkKeys) 54 | 55 | val dimension = ShapeUtils.getPointFromRow(child.execute().first(), column_keys, child, isPoint).coord.length 56 | 57 | if (_indexedRDD == null) { 58 | buildIndex() 59 | } 60 | 61 | private[simba] def buildIndex(): Unit = { 62 | val numShufflePartitions = simbaSession.sessionState.simbaConf.indexPartitions 63 | val maxEntriesPerNode = simbaSession.sessionState.simbaConf.maxEntriesPerNode 64 | val sampleRate = simbaSession.sessionState.simbaConf.sampleRate 65 | val transferThreshold = simbaSession.sessionState.simbaConf.transferThreshold 66 | val dataRDD = child.execute().map(row => { 67 | (ShapeUtils.getPointFromRow(row, column_keys, child, isPoint), row) 68 | }) 69 | 70 | val max_entries_per_node = maxEntriesPerNode 71 | val (partitionedRDD, mbr_bounds) = 72 | STRPartition(dataRDD, dimension, numShufflePartitions, sampleRate, transferThreshold, max_entries_per_node) 73 | 74 | val indexed = partitionedRDD.mapPartitions { iter => 75 | val data = iter.toArray 76 | var index: RTree = null 77 | if (data.length > 0) index = RTree(data.map(_._1).zipWithIndex, max_entries_per_node) 78 | Array(IPartition(data.map(_._2), index)).iterator 79 | }.persist(StorageLevel.MEMORY_AND_DISK_SER) 80 | 81 | val partitionSize = indexed.mapPartitions(iter => iter.map(_.data.length)).collect() 82 | 83 | global_rtree = RTree(mbr_bounds.zip(partitionSize) 84 | .map(x => (x._1._1, x._1._2, x._2)), max_entries_per_node) 85 | indexed.setName(table_name.map(n => s"$n $index_name").getOrElse(child.toString)) 86 | _indexedRDD = indexed 87 | } 88 | 89 | override def newInstance(): IndexedRelation = { 90 | RTreeIndexedRelation(output.map(_.newInstance()), child, table_name, 91 | column_keys, index_name)(_indexedRDD).asInstanceOf[this.type] 92 | } 93 | 94 | override def withOutput(new_output: Seq[Attribute]): IndexedRelation = { 95 | RTreeIndexedRelation(new_output, child, table_name, 96 | column_keys, index_name)(_indexedRDD, global_rtree) 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/CircleSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | 22 | /** 23 | * Created by Zhihao Bai on 16-6-28. 24 | */ 25 | class CircleSuite extends FunSuite{ 26 | test("Circle: intersects and minDist with other Circle"){ 27 | val c1 = Circle(Point(Array(-2.0, 0.0)), 2.0) 28 | val c2 = Circle(Point(Array(-1.5, 0.0)), 0.5) 29 | val c3 = Circle(Point(Array(0.0, 1.0)), 1.0) 30 | val c4 = Circle(Point(Array(1.0, 0.0)), 1.0) 31 | val c5 = Circle(Point(Array(2.0, 1.0)), 1.0) 32 | 33 | assert(c1.intersects(c2)) 34 | assert(c1.intersects(c3)) 35 | assert(c1.intersects(c4)) 36 | assert(!c1.intersects(c5)) 37 | 38 | assert(Math.abs(c1.minDist(c2)) < 1e-8) 39 | assert(Math.abs(c1.minDist(c3)) < 1e-8) 40 | assert(Math.abs(c1.minDist(c4)) < 1e-8) 41 | assert(Math.abs(c1.minDist(c5) - (Math.sqrt(17.0) - 3.0)) < 1e-8) 42 | } 43 | 44 | val c = Circle(Point(Array(0.0, 0.0)), 1.0) 45 | test("Circle: intersects and minDist with Point"){ 46 | val p1 = Point(Array(0.5, 0.0)) 47 | val p2 = Point(Array(1.0, 0.0)) 48 | val p3 = Point(Array(1.5, 0.0)) 49 | 50 | assert(c.intersects(p1)) 51 | assert(c.intersects(p2)) 52 | assert(!c.intersects(p3)) 53 | 54 | assert(Math.abs(c.minDist(p1)) < 1e-8) 55 | assert(Math.abs(c.minDist(p2)) < 1e-8) 56 | assert(Math.abs(c.minDist(p3) - 0.5) < 1e-8) 57 | } 58 | test("Circle: intersects and minDist with LineSegment"){ 59 | val l1 = LineSegment(Point(Array(0.0, 0.0)), Point(Array(1.0, 1.0))) 60 | val l2 = LineSegment(Point(Array(1.0, 0.0)), Point(Array(1.0, 1.0))) 61 | val l3 = LineSegment(Point(Array(2.0, 0.0)), Point(Array(1.0, 1.0))) 62 | 63 | assert(c.intersects(l1)) 64 | assert(c.intersects(l2)) 65 | assert(!c.intersects(l3)) 66 | 67 | assert(Math.abs(c.minDist(l1)) < 1e-8) 68 | assert(Math.abs(c.minDist(l2)) < 1e-8) 69 | assert(Math.abs(c.minDist(l3) - (Math.sqrt(2.0) - 1.0)) < 1e-8) 70 | } 71 | test("Circle: intersects and minDist with MBR"){ 72 | val m1 = MBR(Point(Array(0.0, 0.0)), Point(Array(1.0, 1.0))) 73 | val m2 = MBR(Point(Array(1.0, 0.0)), Point(Array(2.0, 1.0))) 74 | val m3 = MBR(Point(Array(2.0, 0.0)), Point(Array(3.0, 1.0))) 75 | 76 | assert(c.intersects(m1)) 77 | assert(c.intersects(m2)) 78 | assert(!c.intersects(m3)) 79 | 80 | assert(Math.abs(c.minDist(m1)) < 1e-8) 81 | assert(Math.abs(c.minDist(m2)) < 1e-8) 82 | assert(Math.abs(c.minDist(m3) - 1.0) < 1e-8) 83 | } 84 | test("Circle: intersects and minDist with Polygon"){ 85 | val ply1 = Polygon.apply(Array(Point(Array(-1.0, -1.0)), Point(Array(1.0, -1.0)), 86 | Point(Array(0.0, 1.0)), Point(Array(-1.0, -1.0)))) 87 | val ply2 = Polygon.apply(Array(Point(Array(1.0, 0.0)), Point(Array(2.0, 0.0)), 88 | Point(Array(2.0, 1.0)), Point(Array(1.0, 0.0)))) 89 | val ply3 = Polygon.apply(Array(Point(Array(2.0, 0.0)), Point(Array(3.0, 0.0)), 90 | Point(Array(3.0, 1.0)), Point(Array(2.0, 0.0)))) 91 | 92 | assert(c.intersects(ply1)) 93 | assert(c.intersects(ply2)) 94 | assert(!c.intersects(ply3)) 95 | 96 | assert(Math.abs(c.minDist(ply1)) < 1e-8) 97 | assert(Math.abs(c.minDist(ply2)) < 1e-8) 98 | assert(Math.abs(c.minDist(ply3) - 1.0) < 1e-8) 99 | } 100 | 101 | test("Circle: Construct MBR"){ 102 | assert(c.getMBR == MBR(Point(Array(-1.0, -1.0)), Point(Array(1.0, 1.0)))) 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/PointSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | 22 | /** 23 | * Created by Zhihao Bai on 16-6-28. 24 | */ 25 | class PointSuite extends FunSuite{ 26 | val p1 = Point(Array(0.0, 0.0)) 27 | val p2 = Point(Array(1.0, 1.0)) 28 | val p3 = Point(Array(1.0, 1.0)) 29 | 30 | test("Point: intersects with other Point"){ 31 | assert(!p1.intersects(p2)) 32 | assert(p2.intersects(p3)) 33 | } 34 | test("Point: minDist with other point"){ 35 | assert(Math.abs(p1.minDist(p2) - Math.sqrt(2.0)) < 1e-8) 36 | assert(p2.minDist(p3) == 0.0) 37 | } 38 | test("Point: equals another"){ 39 | assert(!(p1 == p2)) 40 | assert(p2 == p3) 41 | } 42 | test("Point: less than another"){ 43 | assert(p1 <= p2) 44 | assert(p2 <= p3) 45 | assert(!(p2 <= p1)) 46 | } 47 | test("Shift one point"){ 48 | assert(p1.shift(1.0) == p2) 49 | } 50 | 51 | 52 | val p = Point(Array(0.0, 0.0)) 53 | 54 | test("Point: intersects and minDist with MBR"){ 55 | val m1 = MBR(Point(Array(-2.0, 1.0)), Point(Array(-1.0, 2.0))) 56 | val m2 = MBR(Point(Array(-1.0, -1.0)), Point(Array(0.0, 0.0))) 57 | val m3 = MBR(Point(Array(-1.0, -1.0)), Point(Array(1.0, 1.0))) 58 | 59 | assert(!p.intersects(m1)) 60 | assert(p.intersects(m2)) 61 | assert(p.intersects(m3)) 62 | 63 | assert(Math.abs(p.minDist(m1) - Math.sqrt(2)) < 1e-8) 64 | assert(Math.abs(p.minDist(m2)) < 1e-8) 65 | assert(Math.abs(p.minDist(m3)) < 1e-8) 66 | } 67 | 68 | test("Point: intersects and minDist with Circle"){ 69 | val c1 = Circle(Point(Array(2.0, 0.0)), 1.0) 70 | val c2 = Circle(Point(Array(1.0, 0.0)), 1.0) 71 | val c3 = Circle(Point(Array(0.0, 0.0)), 1.0) 72 | 73 | assert(!p.intersects(c1)) 74 | assert(p.intersects(c2)) 75 | assert(p.intersects(c3)) 76 | 77 | assert(Math.abs(p.minDist(c1) - 1.0) < 1e-8) 78 | assert(Math.abs(p.minDist(c2)) < 1e-8) 79 | assert(Math.abs(p.minDist(c3)) < 1e-8) 80 | } 81 | test("Point: intersects and minDist with LineSegment"){ 82 | val s1 = LineSegment(Point(Array(1.0, 1.0)), Point(Array(2.0, 1.0))) 83 | val s2 = LineSegment(Point(Array(0.0, 0.0)), Point(Array(1.0, 0.0))) 84 | val s3 = LineSegment(Point(Array(-1.0, 0.0)), Point(Array(1.0, 0.0))) 85 | 86 | assert(!p.intersects(s1)) 87 | assert(p.intersects(s2)) 88 | assert(p.intersects(s3)) 89 | 90 | assert(Math.abs(p.minDist(s1) - Math.sqrt(2.0)) < 1e-8) 91 | assert(Math.abs(p.minDist(s2)) < 1e-8) 92 | assert(Math.abs(p.minDist(s3)) < 1e-8) 93 | } 94 | 95 | test("Point: intersects and minDist with Polygon"){ 96 | val ply1 = Polygon.apply(Array(Point(Array(-1.0, -1.0)), Point(Array(1.0, -1.0)), 97 | Point(Array(0.0, 1.0)), Point(Array(-1.0, -1.0)))) 98 | val ply2 = Polygon.apply(Array(Point(Array(0.0, 0.0)), Point(Array(4.0, 0.0)), 99 | Point(Array(3.0, 2.0)), Point(Array(0.0, 0.0)))) 100 | val ply3 = Polygon.apply(Array(Point(Array(1.0, -1.0)), Point(Array(2.0, 1.0)), 101 | Point(Array(1.0, 1.0)), Point(Array(1.0, -1.0)))) 102 | 103 | assert(p.intersects(ply1)) 104 | // assert(p.intersects(ply2)) 105 | assert(!p.intersects(ply3)) 106 | 107 | assert(Math.abs(p.minDist(ply1)) < 1e-8) 108 | assert(Math.abs(p.minDist(ply2)) < 1e-8) 109 | assert(Math.abs(p.minDist(ply3) - 1.0) < 1e-8) 110 | } 111 | 112 | test("Point: Construct MBR"){ 113 | assert(p.getMBR == MBR(Point(Array(0.0, 0.0)), Point(Array(0.0, 0.0)))) 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/PolygonSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | 22 | /** 23 | * Created by Zhihao Bai on 16-7-3. 24 | */ 25 | class PolygonSuite extends FunSuite{ 26 | test("Polygon: intersects and minDist with other Polygon"){ 27 | val ply1 = Polygon.apply(Array(Point(Array(-1.0, -1.0)), Point(Array(1.0, -1.0)), 28 | Point(Array(0.0, 1.0)), Point(Array(-1.0, -1.0)))) 29 | val ply2 = Polygon.apply(Array(Point(Array(0.0, 0.0)), Point(Array(2.0, 0.0)), 30 | Point(Array(1.0, 2.0)), Point(Array(0.0, 0.0)))) 31 | val ply3 = Polygon.apply(Array(Point(Array(-1.0, 1.0)), Point(Array(1.0, 1.0)), 32 | Point(Array(0.0, 3.0)), Point(Array(-1.0, 1.0)))) 33 | val ply4 = Polygon.apply(Array(Point(Array(1.0, 0.0)), Point(Array(2.0, 0.0)), 34 | Point(Array(2.0, 1.0)), Point(Array(1.0, 0.0)))) 35 | 36 | assert(ply1.intersects(ply2)) 37 | assert(ply1.intersects(ply3)) 38 | assert(!ply1.intersects(ply4)) 39 | 40 | assert(Math.abs(ply1.minDist(ply2)) < 1e-8) 41 | assert(Math.abs(ply1.minDist(ply3)) < 1e-8) 42 | assert(Math.abs(ply1.minDist(ply4) - 1.0 / Math.sqrt(5.0)) < 1e-8) 43 | } 44 | 45 | val ply = Polygon.apply(Array(Point(Array(0.0, 0.0)), Point(Array(2.0, 0.0)), 46 | Point(Array(1.0, 2.0)), Point(Array(0.0, 0.0)))) 47 | 48 | test("Polygon: intersects and minDist with Point"){ 49 | val p1 = Point(Array(1.0, 1.0)) 50 | val p2 = Point(Array(1.5, 1.0)) 51 | val p3 = Point(Array(2.0, 1.0)) 52 | 53 | assert(ply.intersects(p1)) 54 | // assert(ply.intersects(p2)) 55 | assert(!ply.intersects(p3)) 56 | 57 | assert(Math.abs(ply.minDist(p1)) < 1e-8) 58 | assert(Math.abs(ply.minDist(p2)) < 1e-8) 59 | assert(Math.abs(ply.minDist(p3) - 1.0 / Math.sqrt(5.0)) < 1e-8) 60 | } 61 | test("Polygon: intersects and minDist with LineSegment"){ 62 | val s1 = LineSegment(Point(Array(1.0, 1.0)), Point(Array(2.0, 1.0))) 63 | val s2 = LineSegment(Point(Array(0.0, 2.0)), Point(Array(2.0, 2.0))) 64 | val s3 = LineSegment(Point(Array(3.0, 0.0)), Point(Array(3.0, 2.0))) 65 | 66 | assert(ply.intersects(s1)) 67 | assert(ply.intersects(s2)) 68 | assert(!ply.intersects(s3)) 69 | 70 | assert(Math.abs(ply.minDist(s1)) < 1e-8) 71 | assert(Math.abs(ply.minDist(s2)) < 1e-8) 72 | assert(Math.abs(ply.minDist(s3) - 1.0) < 1e-8) 73 | } 74 | test("Polygon: intersects and minDist with Circle"){ 75 | val c1 = Circle(Point(Array(1.0, 1.0)), 1.0) 76 | val c2 = Circle(Point(Array(1.0, -1.0)), 1.0) 77 | val c3 = Circle(Point(Array(4.0, 0.0)), 1.0) 78 | 79 | assert(ply.intersects(c1)) 80 | assert(ply.intersects(c2)) 81 | assert(!ply.intersects(c3)) 82 | 83 | assert(Math.abs(ply.minDist(c1)) < 1e-8) 84 | assert(Math.abs(ply.minDist(c2)) < 1e-8) 85 | assert(Math.abs(ply.minDist(c3) - 1.0) < 1e-8) 86 | } 87 | 88 | test("Polygon: intersects and minDist with MBR"){ 89 | val m1 = MBR(Point(Array(1.0, 1.0)), Point(Array(2.0, 2.0))) 90 | val m2 = MBR(Point(Array(0.0, 2.0)), Point(Array(2.0, 4.0))) 91 | val m3 = MBR(Point(Array(3.0, 0.0)), Point(Array(4.0, 1.0))) 92 | 93 | assert(ply.intersects(m1)) 94 | assert(ply.intersects(m2)) 95 | assert(!ply.intersects(m3)) 96 | 97 | assert(Math.abs(ply.minDist(m1)) < 1e-8) 98 | assert(Math.abs(ply.minDist(m2)) < 1e-8) 99 | assert(Math.abs(ply.minDist(m3) - 1.0) < 1e-8) 100 | } 101 | 102 | test("Polygon: construct MBR"){ 103 | assert(ply.getMBR == MBR(Point(Array(0.0, 0.0)), Point(Array(2.0, 2.0)))) 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/MBRSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | 22 | /** 23 | * Created by Zhihao Bai on 16-7-3. 24 | */ 25 | class MBRSuite extends FunSuite{ 26 | test("MBR: intersects and minDist with other MBR"){ 27 | val m1 = MBR(Point(Array(0.0, 0.0)), Point(Array(2.0, 2.0))) 28 | val m2 = MBR(Point(Array(1.0, 1.0)), Point(Array(3.0, 3.0))) 29 | val m3 = MBR(Point(Array(2.0, 0.0)), Point(Array(3.0, 1.0))) 30 | val m4 = MBR(Point(Array(1.0, 3.0)), Point(Array(2.0, 4.0))) 31 | 32 | assert(m1.intersects(m2)) 33 | assert(m1.intersects(m3)) 34 | assert(!m1.intersects(m4)) 35 | 36 | assert(Math.abs(m1.minDist(m2)) < 1e-8) 37 | assert(Math.abs(m1.minDist(m3)) < 1e-8) 38 | assert(Math.abs(m1.minDist(m4) - 1.0) < 1e-8) 39 | } 40 | 41 | val m = MBR(Point(Array(0.0, 0.0)), Point(Array(2.0, 2.0))) 42 | test("MBR: intersects and minDist with Point"){ 43 | val p1 = Point(Array(1.0,1.0)) 44 | val p2 = Point(Array(2.0, 1.0)) 45 | val p3 = Point(Array(3.0, 0.0)) 46 | 47 | assert(m.intersects(p1)) 48 | assert(m.intersects(p2)) 49 | assert(!m.intersects(p3)) 50 | 51 | assert(Math.abs(m.minDist(p1)) < 1e-8) 52 | assert(Math.abs(m.minDist(p2)) < 1e-8) 53 | assert(Math.abs(m.minDist(p3) - 1.0) < 1e-8) 54 | } 55 | test("MBR: intersects and minDist with LineSegment"){ 56 | val l1 = LineSegment(Point(Array(1.0, 1.0)), Point(Array(3.0, 2.0))) 57 | val l2 = LineSegment(Point(Array(1.0, 3.0)), Point(Array(3.0, 1.0))) 58 | val l3 = LineSegment(Point(Array(3.0, 3.0)), Point(Array(4.0, 2.0))) 59 | 60 | assert(m.intersects(l1)) 61 | assert(m.intersects(l2)) 62 | assert(!m.intersects(l3)) 63 | 64 | assert(Math.abs(m.minDist(l1)) < 1e-8) 65 | assert(Math.abs(m.minDist(l2)) < 1e-8) 66 | assert(Math.abs(m.minDist(l3) - Math.sqrt(2.0)) < 1e-8) 67 | } 68 | test("MBR: intersects and minDist with Circle"){ 69 | val c1 = Circle(Point(Array(2.0, 1.0)), 1.0) 70 | val c2 = Circle(Point(Array(3.0, 3.0)), Math.sqrt(2.0)) 71 | val c3 = Circle(Point(Array(4.0, 1.0)), 1.0) 72 | 73 | assert(m.intersects(c1)) 74 | assert(m.intersects(c2)) 75 | assert(!m.intersects(c3)) 76 | 77 | assert(Math.abs(m.minDist(c1)) < 1e-8) 78 | assert(Math.abs(m.minDist(c2)) < 1e-8) 79 | assert(Math.abs(m.minDist(c3) - 1.0) < 1e-8) 80 | } 81 | test("MBR: intersects and minDist with Polygon"){ 82 | val ply1 = Polygon.apply(Array(Point(Array(-1.0, -1.0)), Point(Array(1.0, -1.0)), 83 | Point(Array(0.0, 1.0)), Point(Array(-1.0, -1.0)))) 84 | val ply2 = Polygon.apply(Array(Point(Array(-2.0, -1.0)), Point(Array(0.0, -1.0)), 85 | Point(Array(0.0, 1.0)), Point(Array(-2.0, -1.0)))) 86 | val ply3 = Polygon.apply(Array(Point(Array(2.0, -1.0)), Point(Array(3.0, -1.0)), 87 | Point(Array(3.0, 0.0)), Point(Array(2.0, -1.0)))) 88 | 89 | assert(m.intersects(ply1)) 90 | assert(m.intersects(ply2)) 91 | assert(!m.intersects(ply3)) 92 | 93 | assert(Math.abs(m.minDist(ply1)) < 1e-8) 94 | assert(Math.abs(m.minDist(ply2)) < 1e-8) 95 | assert(Math.abs(m.minDist(ply3) - Math.sqrt(2.0) / 2.0) < 1e-8) 96 | } 97 | 98 | test("MBR: area"){ 99 | assert(Math.abs(m.area - 4.0) < 1e-8) 100 | } 101 | 102 | test("MBR: ratio"){ 103 | val m1 = MBR(Point(Array(1.0, 1.0)), Point(Array(3.0, 3.0))) 104 | assert(Math.abs(m.calcRatio(m1) - 0.25) < 1e-8) 105 | } 106 | 107 | test("MBR: getMBR"){ 108 | assert(m.getMBR == MBR(Point(Array(0.0, 0.0)), Point(Array(2.0, 2.0)))) 109 | } 110 | } 111 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/spatial/LineSegmentSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.scalatest.FunSuite 21 | /** 22 | * Created by Zhihao Bai on 16-7-2. 23 | */ 24 | class LineSegmentSuite extends FunSuite{ 25 | val s1 = LineSegment(Point(Array(-1.0, 0.0)), Point(Array(1.0, 1.0))) 26 | val s2 = LineSegment(Point(Array(0.0, 0.0)), Point(Array(0.0, 2.0))) 27 | val s3 = LineSegment(Point(Array(1.0, 0.0)), Point(Array(1.0, 2.0))) 28 | val s4 = LineSegment(Point(Array(2.0, 0.0)), Point(Array(2.0, 2.0))) 29 | 30 | test("LineSegment: intersects with other LineSegment"){ 31 | assert(s1.intersects(s2)) 32 | assert(s1.intersects(s3)) 33 | assert(!s1.intersects(s4)) 34 | } 35 | 36 | test("LineSegment: minDist with other LineSegment"){ 37 | assert(s1.minDist(s2) == 0.0) 38 | assert(s1.minDist(s3) == 0.0) 39 | assert(Math.abs(s1.minDist(s4) - 1.0) < 1e-6) 40 | } 41 | 42 | val s = LineSegment(Point(Array(-1.0, 0.0)), Point(Array(1.0, 1.0))) 43 | 44 | test("LineSegment: intersect and minDist with Point"){ 45 | val p1 = Point(Array(0.0, 0.5)) 46 | val p2 = Point(Array(1.0, 1.0)) 47 | val p3 = Point(Array(1.0, 0.0)) 48 | 49 | assert(s.intersects(p1)) 50 | assert(s.intersects(p2)) 51 | assert(!s.intersects(p3)) 52 | 53 | assert(Math.abs(s.minDist(p1)) < 1e-8) 54 | assert(Math.abs(s.minDist(p2)) < 1e-8) 55 | assert(Math.abs(s.minDist(p3) - 2.0 / Math.sqrt(5.0)) < 1e-8) 56 | } 57 | 58 | test("LineSegment: intersect and minDist with MBR"){ 59 | val m1 = MBR(Point(Array(-2.0, -1.0)), Point(Array(0.0, 2.0))) 60 | val m2 = MBR(Point(Array(-2.0, -1.0)), Point(Array(-1.0, 0.0))) 61 | val m3 = MBR(Point(Array(1.0, -1.0)), Point(Array(2.0, 0.0))) 62 | 63 | assert(s.intersects(m1)) 64 | assert(s.intersects(m2)) 65 | assert(!s.intersects(m3)) 66 | 67 | assert(Math.abs(s.minDist(m1)) < 1e-8) 68 | assert(Math.abs(s.minDist(m2)) < 1e-8) 69 | assert(Math.abs(s.minDist(m3) - 2.0 / Math.sqrt(5.0)) < 1e-8) 70 | } 71 | 72 | test("LineSegment: intersect and minDist with Circle"){ 73 | val c1 = Circle(Point(Array(0.0, 0.0)), 1.0) 74 | val c2 = Circle(Point(Array(0.25, 0.0)), Math.sqrt(5.0) / 4.0) 75 | val c3 = Circle(Point(Array(2.0, 0.0)), 1.0) 76 | 77 | assert(s.intersects(c1)) 78 | assert(s.intersects(c2)) 79 | assert(!s.intersects(c3)) 80 | 81 | assert(Math.abs(s.minDist(c1)) < 1e-8) 82 | assert(Math.abs(s.minDist(c2)) < 1e-8) 83 | assert(Math.abs(s.minDist(c3) - (Math.sqrt(2.0) - 1.0)) < 1e-8) 84 | } 85 | 86 | test("LineSegment: intersect and minDist with Polygon"){ 87 | val ply1 = Polygon.apply(Array(Point(Array(-1.0, -1.0)), Point(Array(1.0, -1.0)), 88 | Point(Array(0.0, 1.0)), Point(Array(-1.0, -1.0)))) 89 | val ply2 = Polygon.apply(Array(Point(Array(0.0, -1.0)), Point(Array(1.0, -1.0)), 90 | Point(Array(1.0, 1.0)), Point(Array(0.0, -1.0)))) 91 | val ply3 = Polygon.apply(Array(Point(Array(1.0, 0.0)), Point(Array(2.0, 0.0)), 92 | Point(Array(2.0, 1.0)), Point(Array(1.0, 0.0)))) 93 | 94 | assert(s.intersects(ply1)) 95 | assert(s.intersects(ply2)) 96 | assert(!s.intersects(ply3)) 97 | 98 | assert(Math.abs(s.minDist(ply1)) < 1e-8) 99 | assert(Math.abs(s.minDist(ply2)) < 1e-8) 100 | assert(Math.abs(s.minDist(ply3) - Math.sqrt(2.0) / 2.0) < 1e-8) 101 | } 102 | 103 | test("LineSegment: Construct MBR"){ 104 | val seg = LineSegment(Point(Array(-1.0, 1.0)), Point(Array(1.0, 0.0))) 105 | assert(seg.getMBR == MBR(Point(Array(-1.0, 0.0)), Point(Array(1.0, 1.0)))) 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/BKJSparkR.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.index.RTree 22 | import org.apache.spark.sql.simba.partitioner.MapDPartition 23 | import org.apache.spark.sql.simba.spatial.Point 24 | import org.apache.spark.sql.simba.util.ShapeUtils 25 | import org.apache.spark.rdd.RDD 26 | import org.apache.spark.sql.catalyst.InternalRow 27 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 28 | import org.apache.spark.sql.execution.SparkPlan 29 | 30 | import scala.collection.mutable 31 | import scala.util.Random 32 | 33 | /** 34 | * Created by dong on 1/20/16. 35 | * KNN Join based on Block Nested Loop + Local R-Tree 36 | */ 37 | case class BKJSparkR(left_key: Expression, right_key: Expression, l: Literal, 38 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 39 | override def output: Seq[Attribute] = left.output ++ right.output 40 | 41 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 42 | final val max_entries_per_node = simbaSessionState.simbaConf.maxEntriesPerNode 43 | final val k = l.value.asInstanceOf[Number].intValue() 44 | 45 | private class DisOrdering extends Ordering[(InternalRow, Double)] { 46 | override def compare(x : (InternalRow, Double), y: (InternalRow, Double)): Int = 47 | -x._2.compare(y._2) 48 | } 49 | 50 | override protected def doExecute(): RDD[InternalRow] = { 51 | val tot_rdd = left.execute().map((0, _)).union(right.execute().map((1, _))) 52 | 53 | val tot_dup_rdd = tot_rdd.flatMap {x => 54 | val rand_no = new Random().nextInt(num_partitions) 55 | val ans = mutable.ListBuffer[(Int, (Int, InternalRow))]() 56 | if (x._1 == 0) { 57 | val base = rand_no * num_partitions 58 | for (i <- 0 until num_partitions) 59 | ans += ((base + i, x)) 60 | } else { 61 | for (i <- 0 until num_partitions) 62 | ans += ((i * num_partitions + rand_no, x)) 63 | } 64 | ans 65 | } 66 | 67 | val tot_dup_partitioned = MapDPartition(tot_dup_rdd, num_partitions * num_partitions) 68 | 69 | tot_dup_partitioned.mapPartitions {iter => 70 | var left_data = mutable.ListBuffer[(Point, InternalRow)]() 71 | var right_data = mutable.ListBuffer[(Point, InternalRow)]() 72 | while (iter.hasNext) { 73 | val data = iter.next() 74 | if (data._2._1 == 0) { 75 | val tmp_point = ShapeUtils.getShape(left_key, left.output, data._2._2).asInstanceOf[Point] 76 | left_data += ((tmp_point, data._2._2)) 77 | } else { 78 | val tmp_point = ShapeUtils.getShape(right_key, right.output, data._2._2).asInstanceOf[Point] 79 | right_data += ((tmp_point, data._2._2)) 80 | } 81 | } 82 | 83 | val joined_ans = mutable.ListBuffer[(InternalRow, Array[(InternalRow, Double)])]() 84 | 85 | if (right_data.nonEmpty) { 86 | val right_rtree = RTree(right_data.map(_._1).zipWithIndex.toArray, max_entries_per_node) 87 | left_data.foreach(left => 88 | joined_ans += ((left._2, right_rtree.kNN(left._1, k, keepSame = false) 89 | .map(x => (right_data(x._2)._2, x._1.minDist(left._1))))) 90 | ) 91 | } 92 | 93 | joined_ans.iterator 94 | }.reduceByKey((left, right) => (left ++ right).sortWith(_._2 < _._2).take(k), num_partitions) 95 | .flatMap { now => now._2.map(x => new JoinedRow(now._1, x._1)) } 96 | } 97 | 98 | override def children: Seq[SparkPlan] = Seq(left, right) 99 | } 100 | 101 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/MBR.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.apache.spark.sql.simba.ShapeType 21 | import org.apache.spark.sql.types.SQLUserDefinedType 22 | 23 | /** 24 | * Created by dong on 1/15/16. 25 | * Multi-Dimensional Minimum Bounding Box 26 | */ 27 | @SQLUserDefinedType(udt = classOf[ShapeType]) 28 | case class MBR(low: Point, high: Point) extends Shape { 29 | require(low.dimensions == high.dimensions) 30 | require(low <= high) 31 | 32 | override val dimensions: Int = low.dimensions 33 | 34 | override def intersects(other: Shape): Boolean = { 35 | other match { 36 | case p: Point => contains(p) 37 | case mbr: MBR => intersects(mbr) 38 | case cir: Circle => cir.intersects(this) 39 | case poly: Polygon => poly.intersects(this) 40 | case seg: LineSegment => seg.intersects(this) 41 | } 42 | } 43 | 44 | override def minDist(other: Shape): Double = { 45 | other match { 46 | case p: Point => minDist(p) 47 | case mbr: MBR => minDist(mbr) 48 | case cir: Circle => cir.minDist(this) 49 | case poly: Polygon => poly.minDist(this) 50 | case seg: LineSegment => seg.minDist(this) 51 | } 52 | } 53 | 54 | def this(low_x: Double, low_y: Double, high_x: Double, high_y: Double) { 55 | this(Point(Array(low_x, low_y)), Point(Array(high_x, high_y))) 56 | } 57 | 58 | val centroid = new Point(low.coord.zip(high.coord).map(x => (x._1 + x._2) / 2.0)) 59 | 60 | def intersects(other: MBR): Boolean = { 61 | require(low.coord.length == other.low.coord.length) 62 | for (i <- low.coord.indices) 63 | if (low.coord(i) > other.high.coord(i) || high.coord(i) < other.low.coord(i)) { 64 | return false 65 | } 66 | true 67 | } 68 | 69 | def contains(p: Point): Boolean = { 70 | require(low.coord.length == p.coord.length) 71 | for (i <- p.coord.indices) 72 | if (low.coord(i) > p.coord(i) || high.coord(i) < p.coord(i)) { 73 | return false 74 | } 75 | true 76 | } 77 | 78 | def minDist(p: Point): Double = { 79 | require(low.coord.length == p.coord.length) 80 | var ans = 0.0 81 | for (i <- p.coord.indices) { 82 | if (p.coord(i) < low.coord(i)) { 83 | ans += (low.coord(i) - p.coord(i)) * (low.coord(i) - p.coord(i)) 84 | } else if (p.coord(i) > high.coord(i)) { 85 | ans += (p.coord(i) - high.coord(i)) * (p.coord(i) - high.coord(i)) 86 | } 87 | } 88 | Math.sqrt(ans) 89 | } 90 | 91 | def maxDist(p: Point): Double = { 92 | require(low.coord.length == p.coord.length) 93 | var ans = 0.0 94 | for (i <- p.coord.indices) { 95 | ans += Math.max((p.coord(i) - low.coord(i)) * (p.coord(i) - low.coord(i)), 96 | (p.coord(i) - high.coord(i)) * (p.coord(i) - high.coord(i))) 97 | } 98 | Math.sqrt(ans) 99 | } 100 | 101 | def minDist(other: MBR): Double = { 102 | require(low.coord.length == other.low.coord.length) 103 | var ans = 0.0 104 | for (i <- low.coord.indices) { 105 | var x = 0.0 106 | if (other.high.coord(i) < low.coord(i)) { 107 | x = Math.abs(other.high.coord(i) - low.coord(i)) 108 | } else if (high.coord(i) < other.low.coord(i)) { 109 | x = Math.abs(other.low.coord(i) - high.coord(i)) 110 | } 111 | ans += x * x 112 | } 113 | Math.sqrt(ans) 114 | } 115 | 116 | def area: Double = low.coord.zip(high.coord).map(x => x._2 - x._1).product 117 | 118 | def calcRatio(query: MBR): Double = { 119 | val intersect_low = low.coord.zip(query.low.coord).map(x => Math.max(x._1, x._2)) 120 | val intersect_high = high.coord.zip(query.high.coord).map(x => Math.min(x._1, x._2)) 121 | val diff_intersect = intersect_low.zip(intersect_high).map(x => x._2 - x._1) 122 | if (diff_intersect.forall(_ > 0)) 1.0 * diff_intersect.product / area 123 | else 0.0 124 | } 125 | 126 | override def toString: String = "MBR(" + low.toString + "," + high.toString + ")" 127 | 128 | def getMBR: MBR = this.copy() 129 | } -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/simba/index/RTreeSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | import org.scalatest.FunSuite 21 | import org.apache.spark.sql.simba.spatial.{Circle, MBR, Point, Shape} 22 | 23 | /** 24 | * Created by Zhihao Bai on 16-7-19. 25 | */ 26 | class RTreeSuite extends FunSuite{ 27 | var entries = new Array[(Point, Int)](221) 28 | var cnt = 0 29 | for (i <- -10 to 10){ 30 | for(j <- -10 to 10){ 31 | if(Math.abs(i) + Math.abs(j) <= 10) { 32 | entries(cnt) = (new Point(Array(i, j)), i + j) 33 | cnt = cnt + 1 34 | } 35 | } 36 | } 37 | val rtree = RTree.apply(entries, 5) 38 | 39 | test("RTree: range, simple"){ 40 | val A = Point(Array(0.0, 0.0)) 41 | val B = Point(Array(9.0, 9.0)) 42 | val mbr = MBR(A, B) 43 | val range = rtree.range(mbr) 44 | 45 | for(x <-range){ 46 | assert(mbr.intersects(x._1)) 47 | } 48 | 49 | var count = 0 50 | for (i <- -10 to 10){ 51 | for(j <- -10 to 10){ 52 | if(Math.abs(i) + Math.abs(j) <= 10) { 53 | if(i >= 0 && j >= 0 && i <= 9 && j <= 9) 54 | count = count + 1 55 | } 56 | } 57 | } 58 | assert(range.length == count) 59 | } 60 | 61 | test("RTree: range, complex"){ 62 | val A = Point(Array(0.0, 0.0)) 63 | val B = Point(Array(9.0, 9.0)) 64 | val mbr = MBR(A, B) 65 | 66 | val ans = rtree.range(mbr, 10, 1.0) 67 | assert(ans.isDefined) 68 | val range = ans.get 69 | for(x <-range){ 70 | assert(mbr.intersects(x._1)) 71 | } 72 | var count = 0 73 | for (i <- -10 to 10){ 74 | for(j <- -10 to 10){ 75 | if(Math.abs(i) + Math.abs(j) <= 10) { 76 | if(i >= 0 && j >= 0 && i <= 9 && j <= 9) 77 | count = count + 1 78 | } 79 | } 80 | } 81 | assert(range.length == count) 82 | 83 | val no_ans = rtree.range(mbr, 1, 0.0) 84 | assert(no_ans.isEmpty) 85 | } 86 | 87 | test("RTree: circleRange"){ 88 | val center = Point(Array(6.0, 6.0)) 89 | val radius = 6.0 90 | val circle = Circle(center, radius) 91 | val range = rtree.circleRange(center, radius) 92 | 93 | for(x <-range){ 94 | assert(circle.intersects(x._1)) 95 | } 96 | 97 | var count = 0 98 | for (i <- -10 to 10){ 99 | for(j <- -10 to 10){ 100 | if(Math.abs(i) + Math.abs(j) <= 10) { 101 | if((i - 6) * (i - 6) + (j - 6) * (j - 6) <= 36) 102 | count = count + 1 103 | } 104 | } 105 | } 106 | assert(range.length == count) 107 | } 108 | 109 | test("RTree: circleRangeConj"){ 110 | val A = Point(Array(6.0, 6.0)) 111 | val B = Point(Array(-3.0, 0.0)) 112 | val C = Point(Array(0.0, -5.0)) 113 | val circles = Array[(Point, Double)]((A, 10.0), (B, 3.0), (C, 7.0)) 114 | 115 | val range = rtree.circleRangeConj(circles) 116 | 117 | def touch(p: Shape): Boolean = { 118 | for((q, d) <- circles){ 119 | if(q.minDist(p) > d) false 120 | } 121 | true 122 | } 123 | 124 | for(x <-range){ 125 | assert(touch(x._1)) 126 | } 127 | 128 | var count = 0 129 | for (i <- -10 to 10){ 130 | for(j <- -10 to 10){ 131 | if(Math.abs(i) + Math.abs(j) <= 10) { 132 | if((i - 6) * (i - 6) + (j - 6) * (j - 6) <= 100 133 | && (i + 3) * (i + 3) + (j - 0) * (j - 0) <= 9 134 | && (i - 0) * (i - 0) + (j + 5) * (j + 5) <= 49) 135 | count = count + 1 136 | } 137 | } 138 | } 139 | assert(range.length == count) 140 | } 141 | 142 | test("RTree: kNN"){ 143 | val center = Point(Array(10.0, 9.0)) 144 | val k = 4 145 | val range = rtree.kNN(center, k) 146 | 147 | def minDist(p: Point): Double = range.map(x => x._1.minDist(p)).min 148 | 149 | assert(range.length == 4) 150 | assert(minDist(Point(Array(7.0, 3.0))) < 1e-8) 151 | assert(minDist(Point(Array(6.0, 4.0))) < 1e-8) 152 | assert(minDist(Point(Array(5.0, 5.0))) < 1e-8) 153 | assert(minDist(Point(Array(4.0, 6.0))) < 1e-8) 154 | } 155 | } 156 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/Polygon.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import com.vividsolutions.jts.geom.{Polygon => JTSPolygon} 21 | import com.vividsolutions.jts.geom.{Coordinate, Envelope, GeometryFactory} 22 | import com.vividsolutions.jts.io.{WKBReader, WKBWriter, WKTWriter} 23 | import org.apache.spark.sql.simba.ShapeType 24 | import org.apache.spark.sql.types.SQLUserDefinedType 25 | 26 | /** 27 | * Created by Dong Xie on 3/16/2016. 28 | * Light wraper of JTS Polygon 29 | * Note: Only support up to 2 dimension 30 | */ 31 | @SQLUserDefinedType(udt = classOf[ShapeType]) 32 | case class Polygon(content: JTSPolygon) extends Shape { 33 | override val dimensions = 2 34 | 35 | def this() = { 36 | this(null) 37 | } 38 | 39 | val gf = new GeometryFactory() 40 | 41 | override def minDist(other: Shape): Double = { 42 | other match { 43 | case p: Point => minDist(p) 44 | case mbr: MBR => minDist(mbr) 45 | case cir: Circle => minDist(cir) 46 | case poly: Polygon => minDist(poly) 47 | } 48 | } 49 | 50 | override def intersects(other: Shape): Boolean = { 51 | other match { 52 | case p: Point => contains(p) 53 | case mbr: MBR => intersects(mbr) 54 | case cir: Circle => intersects(cir) 55 | case poly: Polygon => intersects(poly) 56 | } 57 | } 58 | 59 | def contains(p: Point): Boolean = { 60 | require(p.coord.length == 2) 61 | content.contains(gf.createPoint(new Coordinate(p.coord(0), p.coord(1)))) 62 | } 63 | 64 | def intersects(mbr: MBR): Boolean = { 65 | require(mbr.low.coord.length == 2) 66 | val low = new Coordinate(mbr.low.coord(0), mbr.low.coord(1)) 67 | val high = new Coordinate(mbr.high.coord(0), mbr.high.coord(1)) 68 | content.intersects(gf.toGeometry(new Envelope(low, high))) 69 | } 70 | 71 | def intersects(cir: Circle): Boolean = minDist(cir.center) <= cir.radius 72 | 73 | def intersects(poly: Polygon): Boolean = content.intersects(poly.content) 74 | 75 | def intersects(seg: LineSegment): Boolean = { 76 | val start = new Coordinate(seg.start.coord(0), seg.start.coord(1)) 77 | val end = new Coordinate(seg.end.coord(0), seg.end.coord(1)) 78 | content.intersects(gf.createLineString(Array(start, end))) 79 | } 80 | 81 | def minDist(p: Point): Double = { 82 | require(p.coord.length == 2) 83 | content.distance(gf.createPoint(new Coordinate(p.coord(0), p.coord(1)))) 84 | } 85 | 86 | def minDist(mbr: MBR): Double = { 87 | require(mbr.low.coord.length == 2) 88 | val low = new Coordinate(mbr.low.coord(0), mbr.low.coord(1)) 89 | val high = new Coordinate(mbr.high.coord(0), mbr.high.coord(1)) 90 | content.distance(gf.toGeometry(new Envelope(low, high))) 91 | } 92 | 93 | def minDist(cir: Circle): Double = { 94 | val res = minDist(cir.center) - cir.radius 95 | if (res <= 0) 0 96 | else res 97 | } 98 | 99 | 100 | def minDist(poly: Polygon): Double = content.distance(poly.content) 101 | 102 | def minDist(seg: LineSegment): Double = { 103 | val start = new Coordinate(seg.start.coord(0), seg.start.coord(1)) 104 | val end = new Coordinate(seg.end.coord(0), seg.end.coord(1)) 105 | content.distance(gf.createLineString(Array(start, end))) 106 | } 107 | 108 | override def toString: String = new WKTWriter().write(content) 109 | def toWKB: Array[Byte] = new WKBWriter().write(content) 110 | 111 | def getMBR: MBR = { 112 | val envelope = content.getEnvelopeInternal 113 | new MBR(envelope.getMinX, envelope.getMinY, envelope.getMaxX, envelope.getMaxY) 114 | } 115 | } 116 | 117 | object Polygon { 118 | def apply(points: Array[Point]): Polygon = { 119 | require(points.length > 2 && points(0).coord.length == 2) 120 | val gf = new GeometryFactory() 121 | Polygon(gf.createPolygon(points.map(x => new Coordinate(x.coord(0), x.coord(1))))) 122 | } 123 | def fromJTSPolygon(polygon: JTSPolygon): Polygon = new Polygon(polygon) 124 | def fromWKB(bytes: Array[Byte]): Polygon = 125 | new Polygon(new WKBReader().read(bytes).asInstanceOf[JTSPolygon]) 126 | } 127 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/util/KryoShapeSerializer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.util 19 | 20 | import com.esotericsoftware.kryo.io.{Input, Output} 21 | import com.esotericsoftware.kryo.{Kryo, Serializer} 22 | import com.vividsolutions.jts.geom.{CoordinateSequence, GeometryFactory, LinearRing} 23 | import org.apache.spark.sql.simba.spatial._ 24 | 25 | /** 26 | * Created by dongx on 11/15/16. 27 | */ 28 | class KryoShapeSerializer extends Serializer[Shape] { 29 | val gf = new GeometryFactory() 30 | val csFactory = gf.getCoordinateSequenceFactory 31 | 32 | private def getTypeInt(o: Shape): Short = o match { 33 | case _: Point => 0 34 | case _: MBR => 1 35 | case _: Circle => 2 36 | case _: Polygon => 3 37 | case _: LineSegment => 4 38 | case _ => -1 39 | } 40 | 41 | private def writeCoordSequence(output: Output, coords: CoordinateSequence) = { 42 | output.writeInt(coords.size(), true) 43 | for (i <- 0 until coords.size()) { 44 | val coord = coords.getCoordinate(i) 45 | output.writeDouble(coord.getOrdinate(0)) 46 | output.writeDouble(coord.getOrdinate(1)) 47 | } 48 | } 49 | 50 | private def readCoordSequence(input: Input) = { 51 | val n = input.readInt(true) 52 | val coords = csFactory.create(n, 2) 53 | for (i <- 0 until n) { 54 | coords.setOrdinate(i, 0, input.readDouble) 55 | coords.setOrdinate(i, 1, input.readDouble) 56 | } 57 | coords 58 | } 59 | 60 | override def write(kryo: Kryo, output: Output, shape: Shape) = { 61 | output.writeShort(getTypeInt(shape)) 62 | shape match { 63 | case p: Point => 64 | output.writeInt(p.dimensions, true) 65 | p.coord.foreach(output.writeDouble) 66 | case m: MBR => 67 | output.writeInt(m.dimensions, true) 68 | m.low.coord.foreach(output.writeDouble) 69 | m.high.coord.foreach(output.writeDouble) 70 | case c: Circle => 71 | output.writeInt(c.dimensions, true) 72 | c.center.coord.foreach(output.writeDouble) 73 | output.writeDouble(c.radius) 74 | case poly: Polygon => 75 | val content = poly.content 76 | writeCoordSequence(output, content.getExteriorRing.getCoordinateSequence) 77 | val n = content.getNumInteriorRing 78 | output.writeInt(n, true) 79 | for (i <- 0 until n) writeCoordSequence(output, content.getInteriorRingN(i).getCoordinateSequence) 80 | case ls: LineSegment => 81 | ls.start.coord.foreach(output.writeDouble) 82 | ls.end.coord.foreach(output.writeDouble) 83 | } 84 | } 85 | 86 | override def read(kryo: Kryo, input: Input, tp: Class[Shape]): Shape = { 87 | val type_int = input.readShort() 88 | if (type_int == 0) { 89 | val dim = input.readInt(true) 90 | val coords = Array.ofDim[Double](dim) 91 | for (i <- 0 until dim) coords(i) = input.readDouble() 92 | Point(coords) 93 | } else if (type_int == 1) { 94 | val dim = input.readInt(true) 95 | val low = Array.ofDim[Double](dim) 96 | val high = Array.ofDim[Double](dim) 97 | for (i <- 0 until dim) low(i) = input.readDouble() 98 | for (i <- 0 until dim) high(i) = input.readDouble() 99 | MBR(Point(low), Point(high)) 100 | } else if (type_int == 2) { 101 | val dim = input.readInt(true) 102 | val center = Array.ofDim[Double](dim) 103 | for (i <- 0 until dim) center(i) = input.readDouble() 104 | Circle(Point(center), input.readDouble()) 105 | } else if (type_int == 3) { 106 | val exterior_ring = gf.createLinearRing(readCoordSequence(input)) 107 | val num_interior_rings = input.readInt(true) 108 | if (num_interior_rings == 0) Polygon(gf.createPolygon(exterior_ring)) 109 | else { 110 | val interior_rings = Array.ofDim[LinearRing](num_interior_rings) 111 | for (i <- 0 until num_interior_rings) interior_rings(i) = gf.createLinearRing(readCoordSequence(input)) 112 | Polygon(gf.createPolygon(exterior_ring, interior_rings)) 113 | } 114 | } else if (type_int == 4) { 115 | val start = Array.ofDim[Double](2) 116 | val end = Array.ofDim[Double](2) 117 | for (i <- 0 until 2) start(i) = input.readDouble() 118 | for (i <- 0 until 2) end(i) = input.readDouble() 119 | LineSegment(Point(start), Point(end)) 120 | } else null 121 | } 122 | } 123 | 124 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/DJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.index.RTree 22 | import org.apache.spark.sql.simba.partitioner.{MapDPartition, STRPartition} 23 | import org.apache.spark.sql.simba.spatial.Point 24 | import org.apache.spark.sql.simba.util.{NumberUtil, ShapeUtils} 25 | import org.apache.spark.rdd.RDD 26 | import org.apache.spark.sql.catalyst.InternalRow 27 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 28 | import org.apache.spark.sql.execution.SparkPlan 29 | 30 | import scala.collection.mutable 31 | 32 | /** 33 | * Created by dong on 1/20/16. 34 | * Distance Join based on SJMR(Spatial Join MapReduce) 35 | */ 36 | case class DJSpark(left_key: Expression, right_key: Expression, l: Literal, 37 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 38 | override def output: Seq[Attribute] = left.output ++ right.output 39 | 40 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 41 | final val sample_rate = simbaSessionState.simbaConf.sampleRate 42 | final val max_entries_per_node = simbaSessionState.simbaConf.maxEntriesPerNode 43 | final val transfer_threshold = simbaSessionState.simbaConf.transferThreshold 44 | final val r = NumberUtil.literalToDouble(l) 45 | 46 | override protected def doExecute(): RDD[InternalRow] = { 47 | val left_rdd = left.execute().map(row => 48 | (ShapeUtils.getShape(left_key, left.output, row).asInstanceOf[Point], row) 49 | ) 50 | 51 | val right_rdd = right.execute().map(row => 52 | (ShapeUtils.getShape(right_key, right.output, row).asInstanceOf[Point], row) 53 | ) 54 | 55 | val dimension = right_rdd.first()._1.coord.length 56 | 57 | val (left_partitioned, left_mbr_bound) = STRPartition(left_rdd, dimension, num_partitions, 58 | sample_rate, transfer_threshold, max_entries_per_node) 59 | val (right_partitioned, right_mbr_bound) = STRPartition(right_rdd, dimension, num_partitions, 60 | sample_rate, transfer_threshold, max_entries_per_node) 61 | 62 | val right_rt = RTree(right_mbr_bound.zip(Array.fill[Int](right_mbr_bound.length)(0)) 63 | .map(x => (x._1._1, x._1._2, x._2)), max_entries_per_node) 64 | 65 | val left_dup = new Array[Array[Int]](left_mbr_bound.length) 66 | val right_dup = new Array[Array[Int]](right_mbr_bound.length) 67 | 68 | var tot = 0 69 | left_mbr_bound.foreach { now => 70 | val res = right_rt.circleRange(now._1, r) 71 | val tmp_arr = mutable.ArrayBuffer[Int]() 72 | res.foreach {x => 73 | if (right_dup(x._2) == null) right_dup(x._2) = Array(tot) 74 | else right_dup(x._2) = right_dup(x._2) :+ tot 75 | tmp_arr += tot 76 | tot += 1 77 | } 78 | left_dup(now._2) = tmp_arr.toArray 79 | } 80 | 81 | val bc_left_dup = sparkContext.broadcast(left_dup) 82 | val bc_right_dup = sparkContext.broadcast(right_dup) 83 | 84 | val left_dup_rdd = left_partitioned.mapPartitionsWithIndex { (id, iter) => 85 | iter.flatMap {now => 86 | val tmp_list = bc_left_dup.value(id) 87 | if (tmp_list != null) tmp_list.map(x => (x, now)) 88 | else Array[(Int, (Point, InternalRow))]() 89 | } 90 | } 91 | 92 | val right_dup_rdd = right_partitioned.mapPartitionsWithIndex { (id, iter) => 93 | iter.flatMap {now => 94 | val tmp_list = bc_right_dup.value(id) 95 | if (tmp_list != null) tmp_list.map(x => (x, now)) 96 | else Array[(Int, (Point, InternalRow))]() 97 | } 98 | } 99 | 100 | val left_dup_partitioned = MapDPartition(left_dup_rdd, tot).map(_._2) 101 | val right_dup_partitioned = MapDPartition(right_dup_rdd, tot).map(_._2) 102 | 103 | left_dup_partitioned.zipPartitions(right_dup_partitioned) {(leftIter, rightIter) => 104 | val ans = mutable.ListBuffer[InternalRow]() 105 | val right_data = rightIter.toArray 106 | if (right_data.nonEmpty) { 107 | val right_index = RTree(right_data.map(_._1).zipWithIndex, max_entries_per_node) 108 | leftIter.foreach {now => 109 | ans ++= right_index.circleRange(now._1, r) 110 | .map(x => new JoinedRow(now._2, right_data(x._2)._2)) 111 | } 112 | } 113 | ans.iterator 114 | } 115 | } 116 | 117 | override def children: Seq[SparkPlan] = Seq(left, right) 118 | } 119 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/partitioner/QuadTreePartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.partitioner 19 | 20 | import org.apache.spark.{Partitioner, SparkEnv} 21 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 22 | import org.apache.spark.shuffle.sort.SortShuffleManager 23 | import org.apache.spark.sql.catalyst.InternalRow 24 | import org.apache.spark.util.{MutablePair, SizeEstimator} 25 | 26 | import scala.collection.mutable 27 | import org.apache.spark.sql.simba.index.{QuadTree, QuadTreeNode, RTree} 28 | import org.apache.spark.sql.simba.spatial.{MBR, Point} 29 | 30 | /** 31 | * Created by gefei on 16-6-8. 32 | * A multi-demensional Data Partitioner based on QuadTree Algorithm 33 | */ 34 | object QuadTreePartitioner { 35 | def sortBasedShuffleOn: Boolean = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] 36 | 37 | def apply(origin: RDD[(Point, InternalRow)], dimension: Int, est_partition: Int, 38 | sample_rate: Double, transfer_threshold: Long) 39 | : (RDD[(Point, InternalRow)], Array[(MBR, Int)], QuadTree) = { 40 | val rdd = if (sortBasedShuffleOn) { 41 | origin.mapPartitions { iter => iter.map(row => (row._1, row._2.copy())) } 42 | } else { 43 | origin.mapPartitions { iter => 44 | val mutablePair = new MutablePair[Point, InternalRow]() 45 | iter.map(row => mutablePair.update(row._1, row._2.copy())) 46 | } 47 | } 48 | 49 | val part = new QuadTreePartitioner(est_partition, sample_rate, dimension, 50 | transfer_threshold, rdd) 51 | val shuffled = new ShuffledRDD[Point, InternalRow, InternalRow](rdd, part) 52 | (shuffled, part.mbrBound, part.global_index) 53 | } 54 | } 55 | 56 | class QuadTreePartitioner(est_partition: Int, 57 | sample_rate: Double, 58 | dimension: Int, 59 | transfer_threshold: Long, 60 | rdd: RDD[_ <: Product2[Point, Any]]) 61 | extends Partitioner { 62 | private case class Bounds(min: Array[Double], max: Array[Double]) 63 | 64 | require(dimension == 2, "Only two dimensions are supported for a QuadTree") 65 | 66 | // val root_qtree_node: QuadTreeNode = null 67 | 68 | var (mbrBound, partitions, global_index) = { 69 | val (data_bounds, total_size, num_of_records) = { 70 | rdd.aggregate[(Bounds, Long, Int)]((null, 0, 0))((bound, data) => { 71 | val new_bound = if (bound._1 == null) { 72 | new Bounds(data._1.coord, data._1.coord) 73 | } else { 74 | new Bounds(bound._1.min.zip(data._1.coord).map(x => Math.min(x._1, x._2)), 75 | bound._1.max.zip(data._1.coord).map(x => Math.max(x._1, x._2))) 76 | } 77 | (new_bound, bound._2 + SizeEstimator.estimate(data._1), bound._3 + 1) 78 | }, (left, right) => { 79 | val new_bound = { 80 | if (left._1 == null) right._1 81 | else if (right._1 == null) left._1 82 | else { 83 | new Bounds(left._1.min.zip(right._1.min).map(x => Math.min(x._1, x._2)), 84 | left._1.max.zip(right._1.max).map(x => Math.max(x._1, x._2))) 85 | } 86 | } 87 | (new_bound, left._2 + right._2, left._3 + right._3) 88 | }) 89 | } // get the partition bound and the total size of a MBR 90 | 91 | // every node in QuadTree has a threshold of max number of entries 92 | val max_entries_per_node = num_of_records / est_partition * 3 93 | 94 | val seed = System.currentTimeMillis() 95 | val sampled = if (total_size * sample_rate <= transfer_threshold){ 96 | rdd.sample(withReplacement = false, sample_rate, seed).map(_._1).collect() 97 | } 98 | else { 99 | rdd.sample(withReplacement = true, transfer_threshold / total_size, seed).map(_._1).collect() 100 | } 101 | 102 | var count = 0 103 | val tmp_qtree = QuadTree(sampled.zipWithIndex, 104 | (data_bounds.min.head, data_bounds.min(1), data_bounds.max.head, data_bounds.max(1))) 105 | 106 | def searchMBROnQuadTree(node: QuadTreeNode): Array[(MBR, Int)] = { 107 | val ans = mutable.ArrayBuffer[(MBR, Int)]() 108 | if (node.children == null){ 109 | val mbr = new MBR(Point(Array(node.x_low, node.y_low)), 110 | Point(Array(node.x_high, node.y_high))) 111 | ans += (mbr -> count) 112 | node.objects = Array((mbr.centroid.coord(0), mbr.centroid.coord(1), count)) 113 | count += 1 114 | } else for (child <- node.children) ans ++= searchMBROnQuadTree(child) 115 | ans.toArray 116 | } 117 | 118 | val mbrs = searchMBROnQuadTree(tmp_qtree.root) 119 | (mbrs, mbrs.length, tmp_qtree) 120 | } 121 | 122 | val rt = RTree(mbrBound.map(x => (x._1, x._2, 1)), 25) // use the default value is fine 123 | 124 | override def numPartitions: Int = partitions 125 | override def getPartition(key: Any): Int = { 126 | val k = key.asInstanceOf[Point] 127 | rt.circleRange(k, 0.0).head._2 128 | } 129 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/index/Interval.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.index 19 | 20 | import org.apache.spark.sql.catalyst.expressions._ 21 | import org.apache.spark.sql.simba.expression._ 22 | import org.apache.spark.sql.simba.spatial.Point 23 | import org.apache.spark.sql.simba.util.NumberUtil 24 | 25 | import scala.collection.mutable 26 | 27 | /** 28 | * Created by dongx on 11/10/16. 29 | */ 30 | private[simba] class Interval(var min: (Double, Boolean), 31 | var max: (Double, Boolean)) extends Serializable with PredicateHelper { 32 | def this(min_val: Double = 0.0, max_val: Double = 0.0, 33 | left_closed: Boolean = true, right_closed: Boolean = true) { 34 | this((min_val, left_closed), (max_val, right_closed)) 35 | } 36 | 37 | def isNull: Boolean = min._1 > max._1 || (min._1 == max._1 && !(min._2 && max._2)) 38 | 39 | def intersect(other: Interval): Interval = { 40 | val ans = new Interval() 41 | if (!other.isNull) { 42 | if (other.min._1 > max._1 || other.max._1 < min._1) { 43 | ans.max = (ans.min._1 - 1, true) 44 | } else { 45 | ans.min = if (min._1 < other.min._1) other.min else min 46 | ans.max = if (max._1 > other.max._1) other.max else max 47 | } 48 | } else ans.max = (ans.min._1 - 1, true) 49 | ans 50 | } 51 | 52 | override def toString: String = 53 | (if (min._2) "[" else "(") + min._1 + ", " + max._1 + (if (max._2) "]" else ")") 54 | } 55 | 56 | object Interval extends PredicateHelper { 57 | def getLeafInterval(x: Expression): (Interval, Attribute) = { 58 | x match { 59 | case EqualTo(left: NamedExpression, right: Literal) => 60 | val tmp = NumberUtil.literalToDouble(right) 61 | (new Interval(tmp, tmp), left.toAttribute) 62 | case LessThan(left: NamedExpression, right: Literal) => 63 | (new Interval(Double.MinValue, NumberUtil.literalToDouble(right), 64 | left_closed = false, right_closed = false), left.toAttribute) 65 | case LessThanOrEqual(left: NamedExpression, right: Literal) => 66 | (new Interval(Double.MinValue, NumberUtil.literalToDouble(right), 67 | left_closed = false, right_closed = true), left.toAttribute) 68 | case GreaterThan(left: NamedExpression, right: Literal) => 69 | (new Interval(NumberUtil.literalToDouble(right), Double.MaxValue, 70 | left_closed = false, right_closed = false), left.toAttribute) 71 | case GreaterThanOrEqual(left: NamedExpression, right: Literal) => 72 | (new Interval(NumberUtil.literalToDouble(right), Double.MaxValue, 73 | left_closed = true, right_closed = false), left.toAttribute) 74 | case _ => 75 | null 76 | } 77 | } 78 | def conditionToInterval(condition: Expression, column: List[Attribute], dimension: Int) 79 | : (Array[Interval], Array[Expression], Boolean) = { 80 | var all_knn_flag = true 81 | val leaf_nodes = splitConjunctivePredicates(condition) // split AND expression 82 | val intervals: Array[Interval] = new Array[Interval](dimension) 83 | for (i <- 0 until dimension) 84 | intervals(i) = new Interval(Double.MinValue, Double.MaxValue, false, false) 85 | var ans = mutable.ArrayBuffer[Expression]() 86 | leaf_nodes.foreach {now => 87 | val tmp_interval = getLeafInterval(now) 88 | if (tmp_interval != null) { 89 | for (i <- 0 until dimension) 90 | if (column.indexOf(tmp_interval._2) == i) { 91 | intervals(i) = intervals(i).intersect(tmp_interval._1) 92 | } 93 | all_knn_flag = false 94 | } else { 95 | now match { 96 | case range @ InRange(point: Expression, point_low, point_high) => 97 | val low = point_low.asInstanceOf[Literal].value.asInstanceOf[Point].coord 98 | val high = point_high.asInstanceOf[Literal].value.asInstanceOf[Point].coord 99 | for (i <- 0 until dimension) { 100 | intervals(i) = intervals(i).intersect(new Interval(low(i), high(i))) 101 | } 102 | all_knn_flag = false 103 | case knn @ InKNN(point: Expression, target, k: Literal) => 104 | ans += knn 105 | case cr @ InCircleRange(point: Expression, target, r: Literal) => 106 | ans += cr 107 | all_knn_flag = false 108 | case _ => 109 | all_knn_flag = false 110 | } 111 | } 112 | } 113 | (intervals, ans.toArray, all_knn_flag) 114 | } 115 | 116 | def getBoundNumberForInterval(interval: Interval, 117 | range_bounds: Array[Double]): Seq[Int] = { 118 | val res = new mutable.HashSet[Int]() 119 | if (interval != null){ 120 | val start = range_bounds.indexWhere(x => x >= interval.min._1) 121 | var end = range_bounds.indexWhere(x => x >= interval.max._1) 122 | if (end == -1) end = range_bounds.length 123 | if (start >= 0) { 124 | for (i <- start to end + 1) 125 | res.add(i) 126 | } else res.add(range_bounds.length) 127 | } 128 | res.toArray 129 | } 130 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/partitioner/KDTreePartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | package org.apache.spark.sql.simba.partitioner 18 | 19 | import org.apache.spark.{Partitioner, SparkEnv} 20 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 21 | import org.apache.spark.shuffle.sort.SortShuffleManager 22 | import org.apache.spark.sql.catalyst.InternalRow 23 | import org.apache.spark.util.{MutablePair, SizeEstimator} 24 | 25 | import scala.collection.mutable 26 | import org.apache.spark.sql.simba.index.RTree 27 | import org.apache.spark.sql.simba.spatial.{MBR, Point} 28 | 29 | 30 | /** 31 | * Created by gefei on 16-6-8. 32 | * A Multi-Dimensional Data Partitioner based on KDTree Algorithm 33 | */ 34 | object KDTreePartitioner { 35 | def sortBasedShuffleOn: Boolean = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] 36 | 37 | def apply(origin: RDD[(Point, InternalRow)], dimension: Int, est_partition: Int, 38 | sample_rate: Double, transfer_threshold: Long) 39 | : (RDD[(Point, InternalRow)], Array[(MBR, Int)]) = { 40 | val rdd = if (sortBasedShuffleOn) { 41 | origin.mapPartitions { iter => iter.map(row => (row._1, row._2.copy())) } 42 | } else { 43 | origin.mapPartitions { iter => 44 | val mutablePair = new MutablePair[Point, InternalRow]() 45 | iter.map(row => mutablePair.update(row._1, row._2.copy())) 46 | } 47 | } 48 | 49 | val part = new KDTreePartitioner(est_partition, sample_rate, dimension, 50 | transfer_threshold, rdd) 51 | val shuffled = new ShuffledRDD[Point, InternalRow, InternalRow](rdd, part) 52 | (shuffled, part.mbrBound) 53 | } 54 | 55 | } 56 | class KDTreePartitioner(est_partition: Int, 57 | sample_rate: Double, 58 | dimension: Int, 59 | transfer_threshold: Long, 60 | rdd: RDD[_ <: Product2[Point, Any]]) 61 | extends Partitioner { 62 | private case class Bounds(min: Array[Double], max: Array[Double]) 63 | 64 | var (mbrBound, partitions) = { 65 | val (data_bounds, total_size, num_of_records) = { 66 | rdd.aggregate[(Bounds, Long, Int)]((null, 0, 0))((bound, data) => { 67 | val new_bound = if (bound._1 == null) { 68 | new Bounds(data._1.coord, data._1.coord) 69 | } else { 70 | new Bounds(bound._1.min.zip(data._1.coord).map(x => Math.min(x._1, x._2)), 71 | bound._1.max.zip(data._1.coord).map(x => Math.max(x._1, x._2))) 72 | } 73 | (new_bound, bound._2 + SizeEstimator.estimate(data._1), bound._3 + 1) 74 | }, (left, right) => { 75 | val new_bound = { 76 | if (left._1 == null) right._1 77 | else if (right._1 == null) left._1 78 | else { 79 | new Bounds(left._1.min.zip(right._1.min).map(x => Math.min(x._1, x._2)), 80 | left._1.max.zip(right._1.max).map(x => Math.max(x._1, x._2))) 81 | } 82 | } 83 | (new_bound, left._2 + right._2, left._3 + right._3) 84 | }) 85 | } // get the partition bound and the total size of a MBR 86 | 87 | val max_entries_per_node = num_of_records / est_partition 88 | 89 | val seed = System.currentTimeMillis() 90 | val sampled = if (total_size * sample_rate <= transfer_threshold){ 91 | rdd.sample(withReplacement = false, sample_rate, seed).map(_._1).collect() 92 | } 93 | else { 94 | rdd.sample(withReplacement = true, transfer_threshold / total_size, seed).map(_._1).collect() 95 | } 96 | 97 | def recursiveGroupPoint(entries: Array[Point], low_bound: Seq[Double], 98 | high_bound: Seq[Double], cur_dim: Int): Array[MBR] = { 99 | var ans = mutable.ArrayBuffer[MBR]() 100 | val grouped = entries.sortWith((a, b) => 101 | a.coord(cur_dim) < b.coord(cur_dim)).grouped(Math.ceil(entries.length / 2.0).toInt).toArray 102 | val center = grouped(1).head.coord 103 | require(grouped.length == 2) 104 | 105 | val new_high = 0 until dimension map {i => 106 | if (i != cur_dim) high_bound(i) 107 | else center(i) 108 | } 109 | val new_low = 0 until dimension map { i => 110 | if (i != cur_dim) low_bound(i) 111 | else center(i) 112 | } 113 | if (grouped(0).length >= max_entries_per_node){ 114 | ans ++= recursiveGroupPoint(grouped(0), low_bound, 115 | new_high, (cur_dim + 1) % dimension) 116 | } else { 117 | ans += new MBR(new Point(low_bound.toArray.clone()), 118 | new Point(new_high.toArray.clone())) 119 | } 120 | if (grouped(1).length >= max_entries_per_node){ 121 | ans ++= recursiveGroupPoint(grouped(1), new_low, 122 | high_bound, (cur_dim + 1) % dimension) 123 | } else { 124 | ans += new MBR(new Point(new_low.toArray.clone()), 125 | new Point(high_bound.toArray.clone())) 126 | } 127 | ans.toArray 128 | } 129 | 130 | val mbrs = recursiveGroupPoint(sampled, data_bounds.min, data_bounds.max, 0) 131 | (mbrs.zipWithIndex, mbrs.length) 132 | } 133 | 134 | val rt = RTree(mbrBound.map(x => (x._1, x._2, 1)), 25) // the default value is fine 135 | 136 | override def numPartitions: Int = partitions 137 | override def getPartition(key: Any): Int = { 138 | val k = key.asInstanceOf[Point] 139 | rt.circleRange(k, 0.0).head._2 140 | } 141 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/spatial/LineSegment.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.spatial 19 | 20 | import org.apache.spark.sql.simba.ShapeType 21 | import org.apache.spark.sql.types.SQLUserDefinedType 22 | 23 | /** 24 | * Created by dongx on 5/31/16. 25 | * This is a simple implementation for Line Segment. 26 | * Note: Currently, we only support 2D line segments. 27 | */ 28 | @SQLUserDefinedType(udt = classOf[ShapeType]) 29 | case class LineSegment(start: Point, end: Point) extends Shape { 30 | require(start.dimensions == 2 && end.dimensions == 2) 31 | 32 | override val dimensions: Int = start.dimensions 33 | 34 | override def intersects(other: Shape): Boolean = { 35 | other match { 36 | case p: Point => contains(p) 37 | case mbr: MBR => intersects(mbr) 38 | case cir: Circle => intersects(cir) 39 | case poly: Polygon => poly.intersects(this) 40 | case seg: LineSegment => intersects(seg) 41 | } 42 | } 43 | 44 | override def minDist(other: Shape): Double = { 45 | other match { 46 | case p: Point => minDist(p) 47 | case mbr: MBR => minDist(mbr) 48 | case cir: Circle => minDist(cir) 49 | case poly: Polygon => poly.minDist(this) 50 | case seg: LineSegment => minDist(seg) 51 | } 52 | } 53 | 54 | private def orientation(p: Point, q: Point, r: Point): Int = { 55 | val cross = (q.coord(1) - p.coord(1)) * (r.coord(0) - q.coord(0)) - 56 | (q.coord(0) - p.coord(0)) * (r.coord(1) - q.coord(1)) 57 | if (cross == 0) 0 58 | else if (cross > 0) 1 59 | else -1 60 | } 61 | 62 | private def withinBox(check: Point, start: Point, end: Point): Boolean = { 63 | if (check.coord(0) >= Math.min(start.coord(0), end.coord(0)) && 64 | check.coord(0) <= Math.max(start.coord(0), end.coord(0)) && 65 | check.coord(1) >= Math.min(start.coord(1), end.coord(1)) && 66 | check.coord(1) <= Math.max(start.coord(1), end.coord(1))) { 67 | true 68 | } else false 69 | } 70 | 71 | def intersects(l: LineSegment): Boolean = intersects(l.start, l.end) 72 | 73 | private def intersects(p: Point, q: Point): Boolean = { 74 | val o1 = orientation(start, end, p) 75 | val o2 = orientation(start, end, q) 76 | val o3 = orientation(p, q, start) 77 | val o4 = orientation(p, q, end) 78 | if (o1 != o2 && o3 != o4) true 79 | else if (o1 == 0 && withinBox(p, start, end)) true 80 | else if (o2 == 0 && withinBox(q, start, end)) true 81 | else if (o3 == 0 && withinBox(start, p, q)) true 82 | else if (o4 == 0 && withinBox(end, p, q)) true 83 | else false 84 | } 85 | 86 | def contains(l: Point): Boolean = orientation(start, l, end) == 0 && withinBox(l, start, end) 87 | 88 | def intersects(cir: Circle): Boolean = { 89 | minDist(cir.center) <= cir.radius 90 | } 91 | 92 | def intersects(mbr: MBR): Boolean = { 93 | assert(mbr.low.coord.length == 2) 94 | if (intersects(mbr.low, Point(Array(mbr.high.coord(0), mbr.low.coord(1))))) true 95 | else if (intersects(mbr.low, Point(Array(mbr.low.coord(0), mbr.high.coord(1))))) true 96 | else if (intersects(mbr.high, Point(Array(mbr.high.coord(0), mbr.low.coord(1))))) true 97 | else if (intersects(mbr.high, Point(Array(mbr.low.coord(0), mbr.high.coord(1))))) true 98 | else false 99 | } 100 | 101 | def minDist(p: Point): Double = { 102 | require(p.coord.length == 2) 103 | val len = start.minDist(end) 104 | if (len == 0) return p.minDist(start) 105 | var t = ((p.coord(0) - start.coord(0)) * (end.coord(0) - start.coord(0)) 106 | + (p.coord(1) - start.coord(1)) * (end.coord(1) - start.coord(1))) / (len * len) 107 | t = Math.max(0, Math.min(1, t)) 108 | val proj_x = start.coord(0) + t * (end.coord(0) - start.coord(0)) 109 | val proj_y = start.coord(1) + t * (end.coord(1) - start.coord(1)) 110 | p.minDist(Point(Array(proj_x, proj_y))) 111 | } 112 | 113 | def minDist(cir: Circle): Double = { 114 | val centeral_dis = minDist(cir.center) 115 | if (centeral_dis <= cir.radius) 0.0 116 | else centeral_dis - cir.radius 117 | } 118 | 119 | def minDist(l: LineSegment): Double = { 120 | if (intersects(l)) 0.0 121 | else { 122 | Math.min(Math.min(minDist(l.start), minDist(l.end)), 123 | Math.min(l.minDist(start), l.minDist(end))) 124 | } 125 | } 126 | 127 | def minDist(mbr: MBR): Double = { 128 | val s1 = LineSegment(mbr.low, Point(Array(mbr.low.coord(0), mbr.high.coord(1)))) 129 | val s2 = LineSegment(mbr.low, Point(Array(mbr.high.coord(0), mbr.low.coord(1)))) 130 | val s3 = LineSegment(mbr.high, Point(Array(mbr.low.coord(0), mbr.high.coord(1)))) 131 | val s4 = LineSegment(mbr.high, Point(Array(mbr.high.coord(0), mbr.low.coord(1)))) 132 | Math.min(Math.min(minDist(s1), minDist(s2)), Math.min(minDist(s3), minDist(s4))) 133 | } 134 | 135 | override def getMBR: MBR = { 136 | val (low_x, high_x) = if (start.coord(0) < end.coord(0)) { 137 | (start.coord(0), end.coord(0)) 138 | } else { 139 | (end.coord(0), start.coord(0)) 140 | } 141 | 142 | val (low_y, high_y) = if (start.coord(1) < end.coord(1)) { 143 | (start.coord(1), end.coord(1)) 144 | } else { 145 | (end.coord(1), start.coord(1)) 146 | } 147 | 148 | MBR(Point(Array(low_x, low_y)), Point(Array(high_x, high_y))) 149 | } 150 | 151 | override def toString: String = "SEG(" + start.toString + "->" + end.toString + ")" 152 | } 153 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/expression/logicalPredicates.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.expression 19 | 20 | import org.apache.spark.sql.catalyst.InternalRow 21 | import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} 22 | import org.apache.spark.sql.catalyst.expressions.{BinaryOperator, Expression, ExpressionDescription, ImplicitCastInputTypes, NullIntolerant, Predicate, UnaryExpression} 23 | import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType} 24 | 25 | /** 26 | * Created by dongx on 11/13/2016. 27 | */ 28 | object LogicalPredicateHelper { 29 | def hasKNN(x: Expression): Boolean = { 30 | x match { 31 | case now@And(_, _) => now.hasKNN 32 | case now@Or(_, _) => now.hasKNN 33 | case now@Not(_) => now.hasKNN 34 | case InKNN(_, _, _) => true 35 | case _ => false 36 | } 37 | } 38 | } 39 | 40 | @ExpressionDescription( 41 | usage = "_FUNC_ expr - Logical not.") 42 | case class Not(child: Expression) 43 | extends UnaryExpression with Predicate with ImplicitCastInputTypes with NullIntolerant { 44 | 45 | val hasKNN: Boolean = LogicalPredicateHelper.hasKNN(child) 46 | 47 | override def toString: String = s"NOT $child" 48 | 49 | override def inputTypes: Seq[DataType] = Seq(BooleanType) 50 | 51 | protected override def nullSafeEval(input: Any): Any = !input.asInstanceOf[Boolean] 52 | 53 | override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { 54 | defineCodeGen(ctx, ev, c => s"!($c)") 55 | } 56 | 57 | override def sql: String = s"(NOT ${child.sql})" 58 | } 59 | 60 | @ExpressionDescription( 61 | usage = "expr1 _FUNC_ expr2 - Logical AND.") 62 | case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate { 63 | 64 | val hasKNN: Boolean = LogicalPredicateHelper.hasKNN(left) || LogicalPredicateHelper.hasKNN(right) 65 | 66 | override def inputType: AbstractDataType = BooleanType 67 | 68 | override def symbol: String = "&&" 69 | 70 | override def sqlOperator: String = "AND" 71 | 72 | override def eval(input: InternalRow): Any = { 73 | val input1 = left.eval(input) 74 | if (input1 == false) { 75 | false 76 | } else { 77 | val input2 = right.eval(input) 78 | if (input2 == false) { 79 | false 80 | } else { 81 | if (input1 != null && input2 != null) { 82 | true 83 | } else { 84 | null 85 | } 86 | } 87 | } 88 | } 89 | 90 | override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { 91 | val eval1 = left.genCode(ctx) 92 | val eval2 = right.genCode(ctx) 93 | 94 | // The result should be `false`, if any of them is `false` whenever the other is null or not. 95 | if (!left.nullable && !right.nullable) { 96 | ev.copy(code = s""" 97 | ${eval1.code} 98 | boolean ${ev.value} = false; 99 | if (${eval1.value}) { 100 | ${eval2.code} 101 | ${ev.value} = ${eval2.value}; 102 | }""", isNull = "false") 103 | } else { 104 | ev.copy(code = s""" 105 | ${eval1.code} 106 | boolean ${ev.isNull} = false; 107 | boolean ${ev.value} = false; 108 | if (!${eval1.isNull} && !${eval1.value}) { 109 | } else { 110 | ${eval2.code} 111 | if (!${eval2.isNull} && !${eval2.value}) { 112 | } else if (!${eval1.isNull} && !${eval2.isNull}) { 113 | ${ev.value} = true; 114 | } else { 115 | ${ev.isNull} = true; 116 | } 117 | } 118 | """) 119 | } 120 | } 121 | } 122 | 123 | @ExpressionDescription( 124 | usage = "expr1 _FUNC_ expr2 - Logical OR.") 125 | case class Or(left: Expression, right: Expression) extends BinaryOperator with Predicate { 126 | 127 | val hasKNN: Boolean = LogicalPredicateHelper.hasKNN(left) || LogicalPredicateHelper.hasKNN(right) 128 | 129 | override def inputType: AbstractDataType = BooleanType 130 | 131 | override def symbol: String = "||" 132 | 133 | override def sqlOperator: String = "OR" 134 | 135 | override def eval(input: InternalRow): Any = { 136 | val input1 = left.eval(input) 137 | if (input1 == true) { 138 | true 139 | } else { 140 | val input2 = right.eval(input) 141 | if (input2 == true) { 142 | true 143 | } else { 144 | if (input1 != null && input2 != null) { 145 | false 146 | } else { 147 | null 148 | } 149 | } 150 | } 151 | } 152 | 153 | override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { 154 | val eval1 = left.genCode(ctx) 155 | val eval2 = right.genCode(ctx) 156 | 157 | // The result should be `true`, if any of them is `true` whenever the other is null or not. 158 | if (!left.nullable && !right.nullable) { 159 | ev.isNull = "false" 160 | ev.copy(code = s""" 161 | ${eval1.code} 162 | boolean ${ev.value} = true; 163 | if (!${eval1.value}) { 164 | ${eval2.code} 165 | ${ev.value} = ${eval2.value}; 166 | }""", isNull = "false") 167 | } else { 168 | ev.copy(code = s""" 169 | ${eval1.code} 170 | boolean ${ev.isNull} = false; 171 | boolean ${ev.value} = true; 172 | if (!${eval1.isNull} && ${eval1.value}) { 173 | } else { 174 | ${eval2.code} 175 | if (!${eval2.isNull} && ${eval2.value}) { 176 | } else if (!${eval1.isNull} && !${eval2.isNull}) { 177 | ${ev.value} = false; 178 | } else { 179 | ${ev.isNull} = true; 180 | } 181 | } 182 | """) 183 | } 184 | } 185 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/RKJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.index.RTree 22 | import org.apache.spark.sql.simba.partitioner.{MapDPartition, STRPartition} 23 | import org.apache.spark.sql.simba.spatial.{MBR, Point} 24 | import org.apache.spark.sql.simba.util.ShapeUtils 25 | import org.apache.spark.rdd.RDD 26 | import org.apache.spark.sql.catalyst.InternalRow 27 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 28 | import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} 29 | import org.apache.spark.sql.execution.SparkPlan 30 | 31 | import scala.collection.mutable 32 | 33 | /** 34 | * Created by dong on 1/20/16. 35 | * KNN Join based on Two-Level R-Tree Structure 36 | */ 37 | case class RKJSpark(left_key: Expression, right_key: Expression, l: Literal, 38 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 39 | override def output: Seq[Attribute] = left.output ++ right.output 40 | 41 | final val num_partitions = simbaSessionState.simbaConf.joinPartitions 42 | final val sample_rate = simbaSessionState.simbaConf.sampleRate 43 | final val transfer_threshold = simbaSessionState.simbaConf.transferThreshold 44 | final val theta_boost = simbaSessionState.simbaConf.thetaBoost 45 | final val max_entries_per_node = simbaSessionState.simbaConf.maxEntriesPerNode 46 | final val k = l.value.asInstanceOf[Number].intValue() 47 | 48 | override def outputPartitioning: Partitioning = UnknownPartitioning(num_partitions) 49 | 50 | override protected def doExecute(): RDD[InternalRow] = { 51 | val left_rdd = left.execute().map(row => 52 | (ShapeUtils.getShape(left_key, left.output, row).asInstanceOf[Point], row) 53 | ) 54 | 55 | val right_rdd = right.execute().map(row => 56 | (ShapeUtils.getShape(right_key, right.output, row).asInstanceOf[Point], row) 57 | ) 58 | 59 | val right_sampled = right_rdd 60 | .sample(withReplacement = false, sample_rate, System.currentTimeMillis()) 61 | .map(_._1).collect().zipWithIndex 62 | val right_rt = RTree(right_sampled, max_entries_per_node) 63 | val dimension = right_sampled.head._1.coord.length 64 | 65 | val (left_partitioned, left_mbr_bound) = 66 | STRPartition(left_rdd, dimension, num_partitions, sample_rate, 67 | transfer_threshold, max_entries_per_node) 68 | 69 | val dim = new Array[Int](dimension) 70 | var remaining = theta_boost.toDouble 71 | for (i <- 0 until dimension) { 72 | dim(i) = Math.ceil(Math.pow(remaining, 1.0 / (dimension - i))).toInt 73 | remaining /= dim(i) 74 | } 75 | 76 | val refined_mbr_bound = left_partitioned.mapPartitionsWithIndex {(id, iter) => 77 | if (iter.hasNext) { 78 | val data = iter.map(_._1).toArray 79 | def recursiveGroupPoint(entries: Array[Point], cur_dim: Int, until_dim: Int) 80 | : Array[(Point, Double)] = { 81 | val len = entries.length.toDouble 82 | val grouped = entries.sortWith(_.coord(cur_dim) < _.coord(cur_dim)) 83 | .grouped(Math.ceil(len / dim(cur_dim)).toInt).toArray 84 | if (cur_dim < until_dim) grouped.flatMap(now => recursiveGroupPoint(now, cur_dim + 1, until_dim)) 85 | else grouped.map {list => 86 | val min = new Array[Double](dimension).map(x => Double.MaxValue) 87 | val max = new Array[Double](dimension).map(x => Double.MinValue) 88 | list.foreach { now => 89 | for (i <- min.indices) min(i) = Math.min(min(i), now.coord(i)) 90 | for (i <- max.indices) max(i) = Math.max(max(i), now.coord(i)) 91 | } 92 | val mbr = MBR(new Point(min), new Point(max)) 93 | var cur_max = 0.0 94 | list.foreach(now => { 95 | val cur_dis = mbr.centroid.minDist(now) 96 | if (cur_dis > cur_max) cur_max = cur_dis 97 | }) 98 | (mbr.centroid, cur_max) 99 | } 100 | } 101 | recursiveGroupPoint(data, 0, dimension - 1).map(x => (x._1, x._2, id)).iterator 102 | } else Array().iterator 103 | }.collect() 104 | 105 | val theta = new Array[Double](refined_mbr_bound.length) 106 | for (i <- refined_mbr_bound.indices) { 107 | val query = refined_mbr_bound(i)._1 108 | val knn_mbr_ans = right_rt.kNN(query, k, keepSame = false) 109 | theta(i) = knn_mbr_ans.last._1.minDist(query) + (refined_mbr_bound(i)._2 * 2.0) 110 | } 111 | 112 | val bc_theta = sparkContext.broadcast(theta) 113 | 114 | val right_dup = right_rdd.flatMap(x => { 115 | var list = mutable.ListBuffer[(Int, (Point, InternalRow))]() 116 | val set = new mutable.HashSet[Int]() 117 | for (i <- refined_mbr_bound.indices) { 118 | val pid = refined_mbr_bound(i)._3 119 | if (!set.contains(pid) && refined_mbr_bound(i)._1.minDist(x._1) < bc_theta.value(i)) { 120 | list += ((pid, x)) 121 | set += pid 122 | } 123 | } 124 | list 125 | }) 126 | 127 | val right_dup_partitioned = MapDPartition(right_dup, left_mbr_bound.length).map(_._2) 128 | 129 | left_partitioned.zipPartitions(right_dup_partitioned) { 130 | (leftIter, rightIter) => 131 | val ans = mutable.ListBuffer[InternalRow]() 132 | val right_data = rightIter.toArray 133 | if (right_data.length > 0) { 134 | val right_index = RTree(right_data.map(_._1).zipWithIndex, max_entries_per_node) 135 | leftIter.foreach(now => 136 | ans ++= right_index.kNN(now._1, k, keepSame = false) 137 | .map(x => new JoinedRow(now._2, right_data(x._2)._2)) 138 | ) 139 | } 140 | ans.iterator 141 | } 142 | } 143 | 144 | override def children: Seq[SparkPlan] = Seq(left, right) 145 | } 146 | 147 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/simba/execution/join/ZKJSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 by Simba Project 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | */ 17 | 18 | package org.apache.spark.sql.simba.execution.join 19 | 20 | import org.apache.spark.sql.simba.execution.SimbaPlan 21 | import org.apache.spark.sql.simba.partitioner.{MapDPartition, RangeDPartition, RangePartition} 22 | import org.apache.spark.sql.simba.spatial.{Point, ZValue} 23 | import org.apache.spark.sql.simba.util.ShapeUtils 24 | import org.apache.spark.rdd.RDD 25 | import org.apache.spark.sql.catalyst.InternalRow 26 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Literal} 27 | import org.apache.spark.sql.execution.SparkPlan 28 | 29 | import scala.collection.mutable 30 | import scala.util.Random 31 | 32 | /** 33 | * Created by dong on 1/20/16. 34 | * Approximate kNN Join based on Z-Value 35 | */ 36 | case class ZKJSpark(left_key: Expression, right_key: Expression, l: Literal, 37 | left: SparkPlan, right: SparkPlan) extends SimbaPlan { 38 | override def output: Seq[Attribute] = left.output ++ right.output 39 | 40 | val k = l.toString.toInt 41 | // Parameters that set in sqlContext.conf 42 | val num_partition = simbaSessionState.simbaConf.joinPartitions 43 | val num_shifts = simbaSessionState.simbaConf.zknnShiftTimes 44 | 45 | private def genRandomShiftVectors(dimension : Int, shift : Int): Array[Array[Int]] = { 46 | val r = new Random(System.currentTimeMillis) 47 | val ans = Array.ofDim[Int](shift + 1, dimension) 48 | for (i <- 0 to shift) 49 | for (j <- 0 until dimension) { 50 | if (i == 0) ans(i)(j) = 0 51 | else ans(i)(j) = Math.abs(r.nextInt(100)) 52 | } 53 | ans 54 | } 55 | 56 | private def calcShiftArray(point : Point, shift : Array[Int]) : Array[Int] = { 57 | val len = point.coord.length 58 | val ans = new Array[Int](len) 59 | 60 | for (i <- 0 until len) 61 | ans(i) = point.coord(i).toInt + shift(i) 62 | ans 63 | } 64 | 65 | private def binarySearch[T](array: Array[T], func: T => Double, key: Double): Int = { 66 | var left = 0 67 | var right = array.length - 1 68 | while (left < right) { 69 | val mid = (left + right) >> 1 70 | if (func(array(mid)) <= key) { 71 | left = mid + 1 72 | } else right = mid 73 | } 74 | left 75 | } 76 | 77 | def zKNNPerIter(left_rdd: RDD[(Point, InternalRow)], right_rdd: RDD[(Point, InternalRow)], 78 | k: Int, shift: Array[Int]): RDD[(InternalRow, Array[(InternalRow, Double)])] = { 79 | val packed_left_rdd = left_rdd.map(row => 80 | (ZValue(calcShiftArray(row._1, shift)).toDouble, row)) 81 | 82 | val (left_partitioned_rdd, left_rdd_bound) = RangePartition(packed_left_rdd, num_partition) 83 | 84 | val packed_right_rdd = right_rdd.mapPartitions(iter => iter.map(row => 85 | (ZValue(calcShiftArray(row._1, shift)).toDouble, row))) 86 | val right_partitioned_rdd = RangeDPartition(packed_right_rdd, left_rdd_bound) 87 | 88 | 89 | val right_rdd_with_indexed = right_partitioned_rdd.zipWithIndex() 90 | val right_rdd_bound = right_rdd_with_indexed.mapPartitions(iter => new Iterator[(Long, Long)] { 91 | def hasNext = iter.hasNext 92 | def next() = { 93 | val left = iter.next()._2 94 | var right = left 95 | while (iter.hasNext) 96 | right = iter.next()._2 97 | (left - k, right + k) 98 | } 99 | }).collect() 100 | 101 | val right_dup_rdd = right_rdd_with_indexed.flatMap(item => { 102 | var tmp_arr = mutable.ListBuffer[(Int, ((Double, Point), InternalRow))]() 103 | var part = 0 104 | if (right_rdd_bound.length < 128) { 105 | while (part < right_rdd_bound.length && right_rdd_bound(part)._1 <= item._2) { 106 | if (right_rdd_bound(part)._2 >= item._2) { 107 | tmp_arr += ((part, ((item._1._1, item._1._2._1), item._1._2._2.copy()))) 108 | } 109 | part += 1 110 | } 111 | } else { 112 | part = binarySearch[(Long, Long)](right_rdd_bound, _._2.toDouble, item._2.toDouble) 113 | while (part < right_rdd_bound.length && right_rdd_bound(part)._1 <= item._2) { 114 | tmp_arr += ((part, ((item._1._1, item._1._2._1), item._1._2._2.copy()))) 115 | part += 1 116 | } 117 | } 118 | tmp_arr 119 | }) 120 | 121 | val right_dup_partitioned = MapDPartition(right_dup_rdd, right_rdd_bound.length) 122 | .mapPartitions(iter => iter.map(x => (x._2._1._1, (x._2._1._2, x._2._2)))) 123 | 124 | left_partitioned_rdd.zipPartitions(right_dup_partitioned) { (leftIter, rightIter) => { 125 | val tmp_arr = mutable.ListBuffer[(InternalRow, Array[(InternalRow, Double)])]() 126 | val leftArr = leftIter.toArray 127 | val rightArr = rightIter.toArray 128 | for (i <- leftArr.indices) { 129 | var pos = 0 130 | if (rightArr.length < 128) { 131 | while (pos < rightArr.length - 1 && rightArr(pos)._1 <= leftArr(i)._1) pos += 1 132 | } else binarySearch[(Double, (Point, InternalRow))](rightArr, _._1, leftArr(i)._1) 133 | var tmp = Array[(InternalRow, Double)]() 134 | for (j <- (pos - k) until (pos + k)) 135 | if (j >= 0 && j < rightArr.length) { 136 | tmp = tmp :+(rightArr(j)._2._2, leftArr(i)._2._1.minDist(rightArr(j)._2._1)) 137 | } 138 | tmp_arr += (leftArr(i)._2._2 -> tmp.sortWith(_._2 < _._2).take(k)) 139 | } 140 | tmp_arr.iterator 141 | } 142 | } 143 | } 144 | 145 | def doExecute(): RDD[InternalRow] = { 146 | val left_rdd = left.execute().map(row => 147 | (ShapeUtils.getShape(left_key, left.output, row).asInstanceOf[Point], row) 148 | ) 149 | 150 | val right_rdd = right.execute().map(row => 151 | (ShapeUtils.getShape(right_key, right.output, row).asInstanceOf[Point], row) 152 | ) 153 | 154 | val dimension = right_rdd.first._1.coord.length 155 | val shift_vec = genRandomShiftVectors(dimension, num_shifts) 156 | 157 | var joined_rdd = zKNNPerIter(left_rdd, right_rdd, k, shift_vec(0)) 158 | for (i <- 1 to num_shifts) 159 | joined_rdd = joined_rdd.union(zKNNPerIter(left_rdd, right_rdd, k, shift_vec(i))) 160 | 161 | joined_rdd.reduceByKey((left, right) => 162 | (left ++ right).distinct.sortWith(_._2 < _._2).take(k), num_partition).flatMap(now => { 163 | val ans = mutable.ListBuffer[InternalRow]() 164 | now._2.foreach(x => ans += new JoinedRow(now._1, x._1)) 165 | ans 166 | }) 167 | } 168 | 169 | override def children: Seq[SparkPlan] = Seq(left, right) 170 | } 171 | --------------------------------------------------------------------------------