Skip to content

Commit

Permalink
Merge pull request #62 from mayurdb/knnEfficient_takeOrdered
Browse files Browse the repository at this point in the history
Added a KNNEfficient implementation with takeOrdered
  • Loading branch information
JulienPeloton authored Jul 5, 2018
2 parents acf3388 + 6a76b93 commit 288bf87
Show file tree
Hide file tree
Showing 12 changed files with 350 additions and 51 deletions.
4 changes: 4 additions & 0 deletions src/main/scala/com/spark3d/geometryObjects/Shape3D.scala
Original file line number Diff line number Diff line change
Expand Up @@ -101,5 +101,9 @@ object Shape3D extends Serializable {
def hasCenterCloseTo(p: Point3D, epsilon: Double): Boolean = {
center.distanceTo(p) <= epsilon
}

def getHash(): Int = {
(center.getCoordinate.mkString("/") + getEnvelope.toString).hashCode
}
}
}
6 changes: 3 additions & 3 deletions src/main/scala/com/spark3d/spatial3DRDD/Shape3DRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -122,11 +122,11 @@ abstract class Shape3DRDD[T<:Shape3D] extends Serializable {
if (maxItemsPerBox > Int.MaxValue) {
throw new AssertionError(
"""
The max mumber of elements per partition have become greater than Int limit.
Consider increasing number of partitions.
The max number of elements per partition have become greater than Int limit.
Consider increasing the number of partitions.
""")
}
val octree = new Octree(getDataEnvelope, 0, maxItemsPerBox, maxLevels)
val octree = new Octree(getDataEnvelope, 0, null, maxItemsPerBox, maxLevels)
val partitioning = OctreePartitioning.apply(samples, octree)
val grids = partitioning.getGrids
new OctreePartitioner(octree, grids)
Expand Down
118 changes: 96 additions & 22 deletions src/main/scala/com/spark3d/spatialOperator/SpatialQuery.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,34 +19,108 @@ package com.spark3d.spatialOperator
import com.spark3d.geometryObjects.Shape3D.Shape3D
import com.spark3d.utils.GeometryObjectComparator
import org.apache.spark.rdd.RDD
import com.spark3d.spatialPartitioning._

import scala.collection.mutable.PriorityQueue

import scala.collection.mutable
import scala.collection.mutable.{HashSet, ListBuffer, PriorityQueue}
import scala.reflect.ClassTag
import scala.util.control.Breaks._

object SpatialQuery {

def KNN[A <: Shape3D, B <:Shape3D](queryObject: A, rdd: RDD[B], k: Int): List[B] = {

val pq: PriorityQueue[B] = PriorityQueue.empty[B](new GeometryObjectComparator[B](queryObject.center))

val itr = rdd.toLocalIterator

while (itr.hasNext) {
val currentElement = itr.next
if (pq.size < k) {
pq.enqueue(currentElement)
} else {
val currentEleDist = currentElement.center.distanceTo(queryObject.center)
// TODO make use of pq.max
val maxElement = pq.dequeue
val maxEleDist = maxElement.center.distanceTo(queryObject.center)
if (currentEleDist < maxEleDist) {
pq.enqueue(currentElement)
} else {
pq.enqueue(maxElement)
/**
* Finds the K nearest neighbors of the query object. The naive implementation here searches
* through all the the objects in the RDD to get the KNN. The nearness of the objects here
* is decided on the basis of the distance between their centers.
*
* @param queryObject object to which the knn are to be found
* @param rdd RDD of a Shape3D (Shape3DRDD)
* @param k number of nearest neighbors are to be found
* @return knn
*/
def KNN[A <: Shape3D: ClassTag, B <:Shape3D: ClassTag](queryObject: A, rdd: RDD[B], k: Int): List[B] = {
val knn = rdd.takeOrdered(k)(new GeometryObjectComparator[B](queryObject.center))
knn.toList
}

/**
* Much more efficient implementation of the KNN query above. First we seek the partitions in
* which the query object belongs and we will look for the knn only in those partitions. After
* this if the limit k is not satisfied, we keep looking similarly in the neighbors of the
* containing partitions.
*
* @param queryObject object to which the knn are to be found
* @param rdd RDD of a Shape3D (Shape3DRDD)
* @param k number of nearest neighbors are to be found
* @return knn
*/
def KNNEfficient[A <: Shape3D: ClassTag, B <:Shape3D: ClassTag](queryObject: A, rdd: RDD[B], k: Int): List[B] = {

val partitioner = rdd.partitioner.get.asInstanceOf[SpatialPartitioner]
val containingPartitions = partitioner.getPartitionNodes(queryObject)
val containingPartitionsIndex = containingPartitions.map(x => x._1)
val matchedContainingSubRDD = rdd.mapPartitionsWithIndex(
(index, iter) => {
if (containingPartitionsIndex.contains(index)) iter else Iterator.empty
}
)

val knn_1 = matchedContainingSubRDD.takeOrdered(k)(new GeometryObjectComparator[B](queryObject.center))

if (knn_1.size >= k) {
return knn_1.toList
}

val visitedPartitions = new HashSet[Int]
visitedPartitions ++= containingPartitionsIndex

val neighborPartitions = partitioner.getNeighborNodes(queryObject)
.filter(x => !visitedPartitions.contains(x._1)).to[ListBuffer]
val neighborPartitionsIndex = neighborPartitions.map(x => x._1)

val matchedNeighborSubRDD = rdd.mapPartitionsWithIndex(
(index, iter) => {
if (neighborPartitionsIndex.contains(index)) iter else Iterator.empty
}
)

val knn_2 = matchedNeighborSubRDD.takeOrdered(k-knn_1.size)(new GeometryObjectComparator[B](queryObject.center))

var knn_f = knn_1 ++ knn_2
if (knn_f.size >= k) {
return knn_f.toList
}

visitedPartitions ++= neighborPartitionsIndex

breakable {
for (neighborPartition <- neighborPartitions) {
val secondaryNeighborPartitions = partitioner.getSecondaryNeighborNodes(neighborPartition._2, neighborPartition._1)
.filter(x => !visitedPartitions.contains(x._1))
val secondaryNeighborPartitionsIndex = secondaryNeighborPartitions.map(x => x._1)

val matchedSecondaryNeighborSubRDD = rdd.mapPartitionsWithIndex(
(index, iter) => {
if (secondaryNeighborPartitionsIndex.contains(index))
iter
else
Iterator.empty
}
)


val knn_t = matchedNeighborSubRDD.takeOrdered(k-knn_f.size)(new GeometryObjectComparator[B](queryObject.center))

knn_f = knn_f ++ knn_t

if (knn_f.size >= k) {
break
}

visitedPartitions ++= secondaryNeighborPartitionsIndex
neighborPartitions ++= secondaryNeighborPartitions
}
}
pq.toList.sortWith(_.center.distanceTo(queryObject.center) < _.center.distanceTo(queryObject.center))
knn_f.toList
}
}
68 changes: 52 additions & 16 deletions src/main/scala/com/spark3d/spatialPartitioning/Octree.scala
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import scala.collection.mutable.Queue
class Octree(
val box: BoxEnvelope,
val level: Int,
val parentNode: Octree = null,
val maxItemsPerNode: Int = 5,
val maxLevel: Int = 10)
extends Serializable {
Expand Down Expand Up @@ -77,56 +78,56 @@ class Octree(
box.minX, (box.maxX - box.minX) / 2,
box.minY, (box.maxY - box.minY) / 2,
box.minZ, (box.maxZ - box.minZ) / 2),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_L_SE) = new Octree(
BoxEnvelope.apply(
(box.maxX - box.minX) / 2, box.maxX,
box.minY, (box.maxY - box.minY) / 2,
box.minZ, (box.maxZ - box.minZ) / 2),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_L_NW) = new Octree(
BoxEnvelope.apply(
box.minX, (box.maxX - box.minX) / 2,
(box.maxY - box.minY) / 2, box.maxY,
box.minZ, (box.maxZ - box.minZ) / 2),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_L_NE) = new Octree(
BoxEnvelope.apply(
(box.maxX - box.minX) / 2, box.maxX,
(box.maxY - box.minY) / 2, box.maxY,
box.minZ, (box.maxZ - box.minZ) / 2),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_U_SW) = new Octree(
BoxEnvelope.apply(
box.minX, (box.maxX - box.minX) / 2,
box.minY, (box.maxY - box.minY) / 2,
(box.maxZ - box.minZ) / 2, box.maxZ),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_U_SE) = new Octree(
BoxEnvelope.apply(
(box.maxX - box.minX) / 2, box.maxX,
box.minY, (box.maxY - box.minY) / 2,
(box.maxZ - box.minZ) / 2, box.maxZ),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_U_NW) = new Octree(
BoxEnvelope.apply(
box.minX, (box.maxX - box.minX) / 2,
(box.maxY - box.minY) / 2, box.maxY,
(box.maxZ - box.minZ) / 2, box.maxZ),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

children(CHILD_U_NE) = new Octree(
BoxEnvelope.apply(
(box.maxX - box.minX) / 2, box.maxX,
(box.maxY - box.minY) / 2, box.maxY,
(box.maxZ - box.minZ) / 2, box.maxZ),
level + 1, maxItemsPerNode, maxLevel)
level + 1, this, maxItemsPerNode, maxLevel)

}

Expand Down Expand Up @@ -254,9 +255,9 @@ class Octree(
* @param obj input object for which the search is to be performed
* @param data a ListBuffer in which the desired data should be placed when the funct() == true
*/
private def dfsTraverse(func: (Octree, BoxEnvelope) => Boolean, obj: BoxEnvelope, data: ListBuffer[BoxEnvelope]): Unit = {
private def dfsTraverse(func: (Octree, BoxEnvelope) => Boolean, obj: BoxEnvelope, data: ListBuffer[Octree]): Unit = {
if (func(this, obj)) {
data += box
data += this
}

if (!isLeaf) {
Expand Down Expand Up @@ -360,22 +361,57 @@ class Octree(
}

/**
* get all the leaf nodes, which intersect, contain or are contained
* get all the containing Envelopes of the leaf nodes, which intersect, contain or are contained
* by the input BoxEnvelope
*
* @param obj Input object to be checked for the match
* @return list of leafNodes which match the conditions
* @return list of Envelopes of the leafNodes which match the conditions
*/
def getMatchedLeaves(obj: BoxEnvelope): ListBuffer[BoxEnvelope] = {
def getMatchedLeafBoxes(obj: BoxEnvelope): ListBuffer[BoxEnvelope] = {

val matchedLeaves = getMatchedLeaves(obj)
matchedLeaves.map(x => x.box)
}

val matchedLeaves = new ListBuffer[BoxEnvelope]
/**
* get all the containing Envelopes of the leaf nodes, which intersect, contain or are contained
* by the input BoxEnvelope
*
* @param obj Input object to be checked for the match
* @return list of leafNodes which match the conditions
*/
def getMatchedLeaves(obj: BoxEnvelope): ListBuffer[Octree] = {
val matchedLeaves = new ListBuffer[Octree]
val traverseFunct: (Octree, BoxEnvelope) => Boolean = {
(node, obj) => node.isLeaf && (node.box.intersects(obj) ||
node.box.contains(obj) ||
obj.contains(node.box))
node.box.contains(obj) ||
obj.contains(node.box))
}

dfsTraverse(traverseFunct, obj, matchedLeaves)
matchedLeaves
}

/**
* Get the neighbors of this node. Neighbors here are leaf sibling or leaf descendants of the
* siblings.
*
* @param queryNode the box of the the input node to avoid passing same node as neighbor
* @return list of lead neghbors and their index/partition ID's
*/
def getLeafNeighbors(queryNode: BoxEnvelope): List[Tuple2[Int, BoxEnvelope]] = {
val leafNeighbors = new ListBuffer[Tuple2[Int, BoxEnvelope]]
if (parentNode != null){
for (neighbor <- parentNode.children) {
if (!neighbor.box.isEqual(queryNode)) {
if (neighbor.isLeaf) {
leafNeighbors += new Tuple2(neighbor.box.indexID, neighbor.box)
} else {
leafNeighbors ++= neighbor.children(0).getLeafNeighbors(queryNode)
}
}
}
}
leafNeighbors.toList.distinct
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -46,10 +46,65 @@ class OctreePartitioner (octree: Octree, grids : List[BoxEnvelope]) extends Spat

val result = HashSet.empty[Tuple2[Int, T]]
var matchedPartitions = new ListBuffer[BoxEnvelope]
matchedPartitions ++= octree.getMatchedLeaves(spatialObject.getEnvelope)
matchedPartitions ++= octree.getMatchedLeafBoxes(spatialObject.getEnvelope)
for(partition <- matchedPartitions) {
result += new Tuple2(partition.indexID, spatialObject)
}
result.toIterator
}

/**
* Gets the partitions which contain the input object.
*
* @param spatialObject input object for which the containment is to be found
* @return list of Tuple of containing partitions and their index/partition ID's
*/
override def getPartitionNodes[T <: Shape3D](spatialObject: T): List[Tuple2[Int, Shape3D]] = {

var partitionNodes = new ListBuffer[Shape3D]
partitionNodes ++= octree.getMatchedLeafBoxes(spatialObject.getEnvelope)
var partitionNodesIDs = partitionNodes.map(x => new Tuple2(x.getEnvelope.indexID, x))
partitionNodesIDs.toList
}

/**
* Gets the partitions which are the neighbors of the partitions which contain the input object.
*
* @param spatialObject input object for which the neighbors are to be found
* @return list of Tuple of neighbor partitions and their index/partition ID's
*/
override def getNeighborNodes[T <: Shape3D](spatialObject: T): List[Tuple2[Int, Shape3D]] = {
val neighborNodes = new ListBuffer[Tuple2[Int, Shape3D]]
val partitionNodes = octree.getMatchedLeaves(spatialObject.getEnvelope)
for (partitionNode <- partitionNodes) {
neighborNodes ++= partitionNode.getLeafNeighbors(partitionNode.box.getEnvelope)
}
neighborNodes.toList
}

/**
* Gets the partitions which are the neighbors to the input partition. Useful when getting
* secondary neighbors (neighbors to neighbor) of the queryObject.
*
* @param containingNode The boundary of the Node for which neighbors are to be found.
* @param containingNodeID The index/partition ID of the containingNode
* @return list of Tuple of secondary neighbor partitions and their index/partition IDs
*/
override def getSecondaryNeighborNodes[T <: Shape3D](containingNode: T, containingNodeID: Int): List[Tuple2[Int, Shape3D]] = {
val secondaryNeighborNodes = new ListBuffer[Tuple2[Int, Shape3D]]
// get the bounding box
val box = containingNode.getEnvelope
// reduce the bounding box slightly to avoid getting all the neighbor nodes as the containing nodes
val searchBox = BoxEnvelope.apply(box.minX+0.0001, box.maxX-0.0001,
box.minY+0.0001, box.maxY-0.0001,
box.minZ+0.0001, box.maxZ-0.0001)
val partitionNodes = octree.getMatchedLeaves(searchBox.getEnvelope)
// ideally partitionNodes should be of size 1 as the input containingNode is nothing but the
// boundary of a node in the tree.
for (partitionNode <- partitionNodes) {
secondaryNeighborNodes ++= partitionNode.getLeafNeighbors(partitionNode.box.getEnvelope)
}
secondaryNeighborNodes.toList
}

}
Loading

0 comments on commit 288bf87

Please sign in to comment.