You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by danielblazevski <gi...@git.apache.org> on 2015/10/03 21:03:15 UTC

[GitHub] flink pull request: Flink 1745

GitHub user danielblazevski opened a pull request:

    https://github.com/apache/flink/pull/1220

    Flink 1745

    I added a quadtree data structure for the knn algorithm.  @chiwanpark made originally made a pull request for a kNN algorithm, and we coordinated so that I incorporate a tree structure. The quadtree scales very well with the number of training + test points, but scales poorly with the dimension (even the R-tree scales poorly with the dimension). I added a flag that is automatically determines whether or not to use the quadtree. My implementation needed to use the Euclidean or SquaredEuclidean distance since I needed a specific notion of the distance between a test point and a box in the quadtree. I added another test KNNQuadTreeSuite in addition to Chiwan Park's KNNITSuite, since C. Park's parameters will automatically choose the brute-force non-quadtree method.
    
    For more details on the quadtree + how I used it for the KNN query, please see another branch I created that has a README.md:
    https://github.com/danielblazevski/flink/tree/FLINK-1745-devel/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/danielblazevski/flink FLINK-1745

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1220.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1220
    
----
commit c7e5056c6d273f6f0f841f77e0fdd91ca221602d
Author: Chiwan Park <ch...@apache.org>
Date:   2015-06-30T08:41:25Z

    [FLINK-1745] [ml] Add exact k-nearest-neighbor join

commit 9d0c7942c09086324fadb29bdce749683a0d1a7e
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-15T21:49:05Z

    modified kNN test to familiarize with Flink and KNN.scala

commit 611248e57166dc549f86f805b590dd4e45cb3df5
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-15T21:49:17Z

    modified kNN test to familiarize with Flink and KNN.scala

commit 1fd8231ce194b52b5a1bd55bbc5e135b3fa5775b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-16T01:26:57Z

    nightly commit, minor changes:  got the filter to work, working on mapping the training set to include box lables

commit 15d7d2cb308b23e24c43d103b85a76b0e665cbd3
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-22T02:02:51Z

    commit before incporporating quadtree

commit 8f2da8a66516565c59df8828de2715b45397cb7f
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-22T15:49:25Z

    did a basic import of QuadTree and Test; to-do:  modify QuadTree to allow KNN.scala to make use of

commit e1cef2c5aea65c6f204caeff6348e2778231f98d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-22T21:03:04Z

    transfered ListBuffers for objects in leaf nodes to Vectors

commit c3387ef2ef59734727b56ea652fdb29af957d20b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T00:41:29Z

    basic test on 2D unit box seems to work -- need to generalize, e.g. to include automated bounding box

commit 48294ff37a5f800e5111280da5a3c03f4375028d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T15:03:06Z

    had to debug quadtree -- back to testing 2D

commit 6403ba14e240ed8d67a296ac789e7e00dece800d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T15:22:46Z

    Testing 2D looks good, strong improvement in run time compared to brute-force method

commit 426466a40bc2625f390fe0d912f56a346e46c8f8
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T19:04:52Z

    added automated detection of bounding box based on min/max values of both training and test sets

commit c35543b828384aa4ce04d56dfcb3d73db46d1e6d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T00:28:56Z

    added automated radius about test point to define localized neighborhood, result runs.  TO-DO:  Lots of tests

commit 8e2d2e78f8533d4192aebe9b4baa7efbfa5928a5
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T00:54:06Z

    Note for future:  previous commit passed test of Chiwan Park had in intial knn implementation

commit d6fd40cb88d6e198e52c368e829bf7d32d432081
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T01:56:38Z

    Note for future:  previous commit passed 3D version of the test that Chiwan Park had in the intial knn implementation

commit 0ec1f4866157ca073341672e7fe9a50871ac0b7c
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T14:27:20Z

    changed filename of QuadTreeTest to QuadTreeSuite, about to make test more comprehensive and similar format to other Flink tests

commit ac81561cad27b65d158ae08fd0fb15bdb51d1c8b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T19:51:32Z

    refactored testing of QuadTree, and added more tests

commit b17f82d5ce0214617c8dbc4a387410057d6f3832
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T22:49:10Z

    added KNNBenchmark to check runtimes

commit 530565835d4b5934fcac9e0e51105bb669fec9be
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T22:49:17Z

    added KNNBenchmark to check runtimes

commit 1f946cb30450604e92bbd0f5959ce9a60eb4c41b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-25T01:13:00Z

    fixed bug -- in find siblings, needed to search for minimal bounding boxes

commit 22e4eb7b57795ad1ca4392ca1c1a8bdae76afa8e
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-27T03:34:16Z

    added more thorough benchmark files; about to modify  bounding box to only bound training set and modify search for the testing set

commit 3723f6b09ec7d45f6444df70a5f699cbf998a4bb
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-27T03:34:21Z

    added more thorough benchmark files; about to modify  bounding box to only bound training set and modify search for the testing set

commit c41d3e1029bf81a37cf3594f202b904e2d99e3ac
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-28T03:53:12Z

    major simplification in choosing the radius to look at nearby neighbors

commit 7c77ea20fd9e8a0c4a33c81b83187c84b380d6b2
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-28T04:03:59Z

    cleaned up; commit before deleting previous sibling search

commit cf4aa5d75611db19040466040c3d29432cb0e5f7
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-28T14:02:51Z

    added new devel branch to push temp changes on github

commit ec6ddb0a57136075b4f77616e6e48eb5bcc50a11
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T00:17:33Z

    cleaned up; removed comments and a unused method

commit 7ed9926d8207b5f59b4ceb968d7ebd732029f5c3
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T20:28:31Z

    fixed bug in bFiltVect, and renamed to trainingFiltered

commit 4b3bb2ec92396bf754d0d207ffc6853406ce7c39
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T21:05:46Z

    fixed bug:  if there are fewer than maxPerBox total training points, do not do heap construction, just make siblingsQueue = root.objects

commit 1662b38822dfdfbbca272d377fa3b94f8246e9e6
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T22:03:42Z

    added metric to constructor, and added a flag to test whether it is Euclidean or SquaredEuclidean

commit f654b841cc8d91fa861f188469831404c288b227
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T22:48:51Z

    changed name of test that uses the Quadtree along with KNN -- modified from CHiwan Park's test to ensure flag to use Quadtree will pass

commit 7928798281dba5554eeb63df7e67400a42e7a381
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T22:54:25Z

    fixed QuadTree test to conform to using a min-heap

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166223
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    +          }
    +        }
    +        else {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715958
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    --- End diff --
    
    Variable names start with a lower letter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-200480609
  
    @tillrohrmann thanks! I'll polish up knn.md soon


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715388
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    --- End diff --
    
    why is `ret` a `var` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41378859
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    --- End diff --
    
    Why we don't use `Vector` for `minVec` and `maxVec` instead of `ListBuffer`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715880
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    --- End diff --
    
    Isn't this here the termination criterion of the recursion? Shouldn't it be on the top-level of you recursive function, like the `else` branch of the outer-most `if` condition?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by hsaputra <gi...@git.apache.org>.
Github user hsaputra commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-207720551
  
    Well, it seems like Travis like it =)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714199
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    --- End diff --
    
    maybe rename `n` -> `node`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146202529
  
    It sounds weird for me. If the user sets `useQuadTree` to false, the algorithm should not use quadtree. Otherwise if the user sets `useQuadTree` to true, the algorithm should check whether quadtree can be used or not.
    
    I don't think that `ListBuffer` is better than `Vector`. For example, we can implement `partitionBox` like following:
    
    ```scala
    def partitionBox(cPart: Seq[Vector], L: Vector): Seq[Vector] = {
      var next = cPart
      (0 until L.size).foreach { i =>
        next = next.flatMap { v =>
          val (up, down) = (v.copy, v)
          up.update(i, up(i) - L(i) / 4)
          down.update(i, down(i) + L(i) / 4)
    
          Seq(up, down)
        }
      }
    
      next
    }
    ```
    
    There are still some style issues in this PR. I recommend reformatting all codes in this PR using IDE such as IntelliJ IDEA.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41492356
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    +      children = Childrennodes.clone()
    +    }
    +
    +    /**
    +      * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +      * plane through the center of the box keeping the n-th coordinate fixed,
    +      *  then shifting it in the n-th direction up and down
    +      * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param cPart
    +     * @param L
    +     * @param dim
    +     * @return
    +     */
    +    def partitionBox(cPart:ListBuffer[ListBuffer[Double]],L:ListBuffer[Double], dim:Int):
    +    ListBuffer[ListBuffer[Double]]=
    --- End diff --
    
    Formatting. See other ML code for style.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-193964375
  
    Hi @chiwanpark, I modified the tests and corrected the package + import statements, please have a look.  
    
    I will add more details soon



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715606
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    --- End diff --
    
    comment style


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145295999
  
    When looking at the details of the Travis CI build, I found the first error messages to be:
    -------------------------------------------------------------------------  Begin of error
    [ERROR] /home/travis/build/apache/flink/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala:23: error: object DataSetUtils is not a member of package org.apache.flink.api.scala
    
    [INFO] import org.apache.flink.api.scala.DataSetUtils._
    
    [INFO]                                   ^
    
    [ERROR] /home/travis/build/apache/flink/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala:170: error: value zipWithUniqueId is not a member of org.apache.flink.api.scala.DataSet[T]
    
    -------------------------------------------------------------------------------------- end of errors
    
    There are additional errors that follow, but those are the first two errors that popped up.  The import of DataSetUtils._ was adopted from @chiwanpark, maybe he has an idea of what is the cause of the error.   I unfortunately was not using Travis CI from the start, but will try to use it from now on.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715261
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    --- End diff --
    
    Are you aware that the caller of this method can now modify the underlying `objects` collection? Is this intended?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-159323661
  
    Cool, great to hear :-) Looking forward to merging this PR and reviewing your approximate kNN implementation.
    
    It's interesting how much faster the approximate solution is gonna be.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-200854252
  
    @tillrohrmann @chiwanpark done, polished up KNN.scala and some minor changes -- e.g. expanding the description of the parameters in the beginning of KNN.scala.  
    
    Looking forward to doing the approximate version.  I ran some tests last week of the pure Scala z-value KNN and it looks promising (https://github.com/danielblazevski/zknn-scala)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086837
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    --- End diff --
    
    Done -- removed casting


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093466
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    --- End diff --
    
    How about checking type of `DistanceMetric` in `QuadTree` constructor?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-209854438
  
    Hi @danielblazevski, sorry for late reply. I checked your updated PR but your last commit (d6f90ce) seems wrong. The commit removes KNN.scala, QuadTree.scala, KNNITSuite.scala, and QuadTreeSuite.scala. Could you check again?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47168084
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092399
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    --- End diff --
    
    To avoid use `var`, we should rewrite this method like following:
    
    ```scala
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #1220: [FLINK-1745] Add exact k-nearest-neighbours algorithm to ...

Posted by thvasilo <gi...@git.apache.org>.
Github user thvasilo commented on the issue:

    https://github.com/apache/flink/pull/1220
  
    Oh, sorry should have checked the master docs.
    
    On Jul 26, 2016 5:29 AM, "Chiwan Park" <no...@github.com> wrote:
    
    > Hi @thvasilo <https://github.com/thvasilo>, I've checked the FlinkML
    > index page
    > <https://ci.apache.org/projects/flink/flink-docs-master/apis/batch/libs/ml/index.html#supervised-learning>.
    > There is a link to k-NN page in the index page. We find the link in the
    > page for Flink 1.1 only because k-NN is merged to only master branch.
    >
    > \u2014
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/flink/pull/1220#issuecomment-235252716>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/AIoToxneJBOuEIqELaXDdf4uoGPPoHEgks5qZf2MgaJpZM4GId-u>
    > .
    >



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63733983
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    --- End diff --
    
    Hmm going quickly over the different distance metric implementations, I think all except for the Cosine and the Tanimoto distance metric should work by simply setting the coordinate to 0 if the vector is inside of the box for this dimension. But I'm not 100% sure either.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-222482793
  
    @danielblazevski I'm using a custom configuration of IntelliJ code formatter. There is a pending [pull request](https://github.com/apache/flink/pull/1963/files) about code formatting in IntelliJ. This might be helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63701460
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    --- End diff --
    
    How did you end up with this formula? If I transform the formula in your comment I would end up with the following: `dim*log(4)+log(log(ntrain))<log(ntrain)`. Is there an error in the formula in the comment?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397325
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,146 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    +$$
    +
    +The brute-force approach is to compute the distance between every training and testing point.  To ease the brute-force computation of computing the distance between every traning point a quadtree is used.  The quadtree scales well in the number of training points, though poorly in the spatial dimension.  The algorithm will automatically choose whether or not to use the quadtree, though the user can override that decision by setting a parameter to force use or not use a quadtree. 
    +
    +##Operations
    +
    +`KNN` is a `Predictor`. 
    +As such, it supports the `fit` and `predict` operation.
    +
    +### Fit
    +
    +KNN is trained given a set of `LabeledVector`:
    +
    +* `fit: DataSet[LabeledVector] => Unit`
    +
    +### Predict
    +
    +KNN predicts for all subtypes of FlinkML's `Vector` the corresponding class label:
    +
    +* `predict[T <: Vector]: DataSet[T] => DataSet[(T, Array[Vector])]`, where the `(T, Array[Vector])` tuple
    +  corresponds to (testPoint, K-nearest training points)
    +
    +## Paremeters
    +The KNN implementation can be controlled by the following parameters:
    +
    +   <table class="table table-bordered">
    +    <thead>
    +      <tr>
    +        <th class="text-left" style="width: 20%">Parameters</th>
    +        <th class="text-center">Description</th>
    +      </tr>
    +    </thead>
    +
    +    <tbody>
    +      <tr>
    +        <td><strong>K</strong></td>
    +        <td>
    +          <p>
    +            Defines the number of nearest-neoghbors to search for.  That is, for each test point, the algorithm finds the K nearest neighbors in the training set
    +            (Default value: <strong>5</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong> DistanceMetric</strong></td>
    +        <td>
    +          <p>
    +            Sets the distance metric we use to calculate the distance between two points. If no metric is specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +            (Default value: <strong> EuclideanDistanceMetric </strong>)
    --- End diff --
    
    Please remove space before EuclideanDistanceMetric and after it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714309
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    --- End diff --
    
    whitespaces `if (n.objects.length < maxPerBox)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #1220: [FLINK-1745] Add exact k-nearest-neighbours algorithm to ...

Posted by thvasilo <gi...@git.apache.org>.
Github user thvasilo commented on the issue:

    https://github.com/apache/flink/pull/1220
  
    Hello Daniel, sorry to bring this up months later,but I see that while the documentation exists, there is nothing linking to it from the FlinkML index page. Would you care to create a new PR linking to the docs from the FlinkML docs landing page? Feel free to create an unsupervised learning category for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47167957
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086773
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    --- End diff --
    
    @tillrohrmann , I see you point, but I'm curious how we should implement this.  For example, I could not quickly find how to determine the size of a `DataSet` here:
    https://ci.apache.org/projects/flink/flink-docs-master/apis/programming_guide.html#broadcast-variables
    
    And then what a good threshold would be for determining if a `DataSet` is small?  Say 10MB?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145353515
  
    I just reviewed quickly. I have some comments for this pull request.
    
    1. This pull request cannot pass checking Scala code style. The Scala code should not exceed 100 characters in each line. I also found some wrong indentations such as line 200-211 in `KNN.scala` file. Please reformat your code.
    2. The meaning of `BruteOrQuad` is ambiguous. From my understand, if `BruteOrQuad` is true, we use quadtree implementation, right? So the other name such as `useQuadTree` would be better.
    3. Please remove blank line after Scala doc comments.
    4. Your implementation assumes that the type of given `Vector` is `DenseVector`. But basically, there is no guarantee of the type of `Vector`. We should generalize the implementation.
    5. We should be able to configure the use of quadtree implementation. Please add configure parameter into `KNN` object.
    6. Currently, `predictDataSet` is too big and complex because of large code in `mapPartition` operation. How about split the operation into `mapPartition` for raw and quadtree?
    7. Please avoid calculation of `BigDecimal`. Maybe we can check whether we use quadtree or not without `BigDecimal` like following:
      * `(4 ^ dim) * Ntest * log(Ntrain) < Ntest * Ntrain` is same as `dim + log_4(log(Ntrain)) < log_4(Ntrain)`
    
    This is great start! If you have some question about my comments, please feel free to reply.
    
    P.S.: How about create training quadtree before cross operation? Is there any intention to create quadtree after cross operation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086839
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45619299
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    --- End diff --
    
    if one of the `DataSets` is small, it might make sense to broadcast the smaller one. That way it wouldn't blow up the size of the intermediate result.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093688
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    --- End diff --
    
    Please add a space after `=`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093319
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    --- End diff --
    
    Please add spaces around `*`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086838
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086745
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    --- End diff --
    
    Indeed it is @tillrohrmann , but I'm not exactly sure what you are suggesting to change.  This is a rare case where one has to check both if the node + any of it's children are leaf nodes before going deeper into the recursion.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-160361394
  
    @tillrohrmann addressed all but (1) everything but broadcasting a small `DataSet` -- I asked some questions in your comment about this and (2) You also had a question about `searchRecurSiblingQueue` that I didn't 100% understand so I clarified a bit directly responding above. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086814
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    --- End diff --
    
    Done and Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145364401
  
    Thanks @chiwanpark for the very useful comments.  I have made changes to the comments, which can be found here:
    https://github.com/danielblazevski/flink/tree/FLINK-1745/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn
    
    I also changed the testing of KNN + QuadTree, which can be found here:
    https://github.com/danielblazevski/flink/tree/FLINK-1745/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/nn
    
    Since useQuadTree is now a parameter, I did not need KNNQuadTreeSuite anymore and I removed it.
    
    I did not address comment 6 yet.  I need to have the training set before I can define a non-user specified useQuadTree, so any main if(useQuadTree) should come within ` val crossed = trainingSet.cross(inputSplit).mapPartition {`
    
    About your last "P.S" comment,  Creating the quadtree after the cross operation is likely more efficient -- each CPU/Node will form their own quadtree, which is what is suggested for the R-tree here:
    https://www.cs.utah.edu/~lifeifei/papers/mrknnj.pdf
    
    This will result less communication overhead than creating a more global quadtree, if that is what you were referring to.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086818
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145350740
  
    You should use the import `org.apache.flink.api.scala.utils._` instead of the `DataSetUtils` one. This was changed in a commit a few weeks back.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r43648312
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Breeze,Vector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[Vector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    parameters.add(useQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object useQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: Vector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +        instance: KNN,
    +        fitParameters: ParameterMap,
    +        input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[Vector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: Vector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (Vector, Array[Vector])] {
    +      override def predictDataSet(
    +          instance: KNN,
    +          predictParameters: ParameterMap,
    +          input: DataSet[T]): DataSet[(Vector, Array[Vector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(Vector, Vector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(Vector, Vector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  var MinArr =  List.range(0,training.values.head.size).toArray
    +                  var MaxArr =  List.range(0,training.values.head.size).toArray
    +
    +                  var trainingFiltered = new ListBuffer[Vector]
    --- End diff --
    
    thanks @chiwanpark, fixed both the `var` and capitalization of `useQuadTree`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r43647151
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Breeze,Vector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[Vector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    parameters.add(useQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object useQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: Vector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +        instance: KNN,
    +        fitParameters: ParameterMap,
    +        input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[Vector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: Vector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (Vector, Array[Vector])] {
    +      override def predictDataSet(
    +          instance: KNN,
    +          predictParameters: ParameterMap,
    +          input: DataSet[T]): DataSet[(Vector, Array[Vector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(Vector, Vector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(Vector, Vector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  var MinArr =  List.range(0,training.values.head.size).toArray
    +                  var MaxArr =  List.range(0,training.values.head.size).toArray
    +
    +                  var trainingFiltered = new ListBuffer[Vector]
    --- End diff --
    
    I think we don't need `var` with mutable data structure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-163441400
  
    you must have messed up when rebasing or something. looking at your history, you have a few of your own commits, then a whole bunch of commits that were already merged, and then a few others of yourself. you're gonna have to re-rebase and clean that up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46121226
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    --- End diff --
    
    It's true. To do it automatically it's tricky. What one could do is either counting both data sources and then decide on the number of elements. Or one could implement a way for the user to specify which side is larger. Like it's done for the join operator with `joinWithTiny` and `joinWithLarge`.
    
    I actually only commented on it because `cross` can be a really costly operation because both sides of the operator have to be effectively shuffled.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45713305
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    --- End diff --
    
    So `minDist` is the sum of the distance squares? How can you compare the distance square with `radius`? Shouldn't radius be squared as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153096308
  
    I would suggest logic like following:
    
    ```scala
    val useQuadTree = ~~~
    
    if (useQuadTree) {
      knnQueryWithQuadTree(training, testing, out)
    } else {
      knnQueryBasic(training, testing, out)
    }
    ```
    
    Or to reduce duplicated code in L257-L266, we can use following:
    
    ```scala
    val useQuadTree = ~~~
    val quadTree: Option[QuadTree] = if (useQuadTree) {
      Some(buildQuadTree(training, testing))
    } else {
      None
    }
    
    for (a <- testing.values) {
      val trainingFiltered: Seq[Vector] = quadTree match {
        case Some(tree) => getSibilingsFromQuadTree(a, tree)
        case None => training.values
      }
    
      for (b <- trainingFiltered) {
        // (training vector, input vector, input key, distance)
        queue.enqueue((b, a._2, a._1, metric.distance(b, a._2)))
        if (queue.size > k) {
          queue.dequeue()
        }
      }
      for (v <- queue) {
        out.collect(v)
      }
    }
    ```
    
    In this case, we create methods about quadtree operation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-199648335
  
    Hi @danielblazevski, thanks for update! Looks good to me for implementation. (Some minor issues and rebasing will be addressed by me.)
    
    About docs, I meant we need to add description, examples and meaning of parameters to documentation in our homepage (`docs/libs/ml/knn.md`).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46120801
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    --- End diff --
    
    You could zip `queryPoint`, `center` and `width` and the map over it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397870
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    if (UseQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                      knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
    --- End diff --
    
    Please re-indent this line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-196879970
  
    @chiwanpark for the docs, it looks like you use an auto-generated markdown file from the source code?  i.e. looks like something analogous to `scaladoc myScalaFile.scala` but for markdown.  The `Parameters` section in the ml-docs seems especially auto-generated


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145293328
  
    The Travis CI build failed.  Not sure what went wrong here. How I worked with Flink was I cloned a fork, built using Maven, then imported to IntelliJ using the directions here:
    https://github.com/apache/flink/blob/master/docs/internals/ide_setup.md
    
    Included already was @chiwanpark 's KNN.scala file, so I only really added QuadTree.scala and two additional tests, which all ran in IntelliJ.  I will try to debug this issue, but am happy to hear anyone's thoughts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45618885
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    --- End diff --
    
    whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41992363
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    --- End diff --
    
    Hi @danielblazevski, sorry for not being response either. We just had our first conference.
    
    You're right that with the current `Vector` interface of FlinkML you cannot do much if you want to apply algebraic operations. The recommended way to do it at the moment is to convert Flink `Vectors` into `BreezeVectors` which give you all the operations you can think of. It is very convenient to convert from and to `BreezeVector`. Simply add `import org.apache.flink.ml.math.Breeze._` to your sources and then you can call `vector.asBreeze` to obtain a `BreezeVector` from a `Vector` and `breezeVector.fromBreeze` to obtain a `Vector` from a `BreezeVector`. Then you can simply perform all these algebraic operations without having to operate on the underlying data structure.
    
    One thing to note is that the conversion is not so costly, because we use the same data representation as breeze for our vectors. Therefore, the object creation is rather cheap. However, it is better to not convert in every step from and to breeze. It's better to convert once to breeze in the beginning and then in the end convert back to Flink. You can take a look at the `SVM` code to see how we used it there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086812
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715156
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    --- End diff --
    
    maybe we could rename it into `nodeObjects` or `nodeElements` since `objects` is kind of generic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166308
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092526
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    --- End diff --
    
    I think we can make this more scalaesque.
    
    ```scala
    def minDist(queryPoint: Vector): Double = {
      val minDist = (0 until queryPoint.size).map { i =>
        if (queryPoint(i) < center(i) - width(i) / 2) {
          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
        } else if (queryPoint(i) > center(i) + width(i) / 2) {
          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
        } else {
          0
        }
      }.sum
    
      distMetric match {
        case _: SquaredEuclideanDistanceMetric => minDist
        case _: EuclideanDistanceMetric => math.sqrt(minDist)
        case _ => throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
      }
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47169693
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41378136
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,298 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.DataSetUtils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[Vector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    parameters.add(useQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object useQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    --- End diff --
    
    Maybe we need default value for this parameter?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41492452
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    +      children = Childrennodes.clone()
    +    }
    +
    +    /**
    +      * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +      * plane through the center of the box keeping the n-th coordinate fixed,
    +      *  then shifting it in the n-th direction up and down
    +      * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param cPart
    +     * @param L
    +     * @param dim
    +     * @return
    +     */
    +    def partitionBox(cPart:ListBuffer[ListBuffer[Double]],L:ListBuffer[Double], dim:Int):
    +    ListBuffer[ListBuffer[Double]]=
    +    {
    +      if (L.length == 1){
    +
    +        var cPartDown = cPart.clone()
    +        //// shift center up and down
    +        val cPartUp = cPart.map{v => v.patch(dim-1, Seq(v(dim - 1) + L(dim-1)/4), 1)}
    +        cPartDown = cPartDown.map{v => v.patch(dim-1, Seq(v(dim - 1) - L(dim-1)/4), 1)}
    +
    +        return cPartDown ++ cPartUp
    --- End diff --
    
    returns are not very scalaesque. Why not putting the following code in an `else` branch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45713377
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    --- End diff --
    
    What does this method do?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092387
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    --- End diff --
    
    We can make this function more simple.
    
    ```scala
    def isNear(queryPoint: Vector, radius: Double): Boolean = {
      minDist(queryPoint) < radius
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086832
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086755
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    --- End diff --
    
    Removed subOne -- only used in defining nodeQueue and it's not needed there, added Scaladocs to searchNeighborsQueue.  Cleaned whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-148091245
  
    Don't worry @danielblazevski. The warning simply says that the netlib blas library cannot find a native implementation (fortran blas routines) on your system. But it then uses the fallback implementation shipping with the netlib blas jar. If you run an example, then you should see that it produces some results.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145352975
  
    Your changes are mostly new files, so you should be able to just rebase on the latest master without any problems.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166203
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63714951
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    --- End diff --
    
    Indeed, nice!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63701881
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric, queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](
    +    training: Vector[T],
    +    testing: Vector[(Long, T)],
    +    k: Int, metric: DistanceMetric,
    --- End diff --
    
    formatting


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63709829
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    --- End diff --
    
    Yup good point.  I think Chiwan made this suggestion to go from the main criterion to a simplified one, I either typed it in wrong or didn't double check, my bad. 
    
    Fixed it (will commit all these changes soon)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63712701
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric, queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](
    +    training: Vector[T],
    +    testing: Vector[(Long, T)],
    +    k: Int, metric: DistanceMetric,
    +    queue: mutable.PriorityQueue[(FlinkVector,
    +      FlinkVector, Long, Double)],
    +    out: Collector[(FlinkVector,
    +      FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr = Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).max + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).max + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k, 20).max
    --- End diff --
    
    agreed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-220083900
  
    The PR looks good to me. The only think which could be good to get rid of is the requirement that you have to select a Euclidean distance for the quadtree. Maybe there is some other characteristic for a distance measure which says whether it's applicable for quadtrees or not. Then we could introduce a new distance metric type to make sure that only appropriate distance measures are used. But this should not be a blocker for merging this PR. 
    
    Thanks for your contribution @danielblazevski. Really good work :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166281
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k,20).max
    +    val trainingQuadTree = new QuadTree(MinVec, MaxVec, metric, maxPerBox)
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for  ((id, vector) <- testing) {
    +      //  Find siblings' objects and do local kNN there
    +      val siblingObjects =
    +        trainingQuadTree.searchNeighborsSiblingQueue(
    +          vector.asInstanceOf[FlinkVector])
    +
    +      // do KNN query on siblingObjects and get max distance of kNN
    +      // then rad is good choice for a neighborhood to do a refined
    +      // local kNN search
    +      val knnSiblings = siblingObjects.map(v => metric.distance(vector, v)
    +      ).sortWith(_ < _).take(k)
    +
    +      val rad = knnSiblings.last
    +      var trainingFiltered = new ListBuffer[FlinkVector]
    +      trainingFiltered =
    +        trainingQuadTree.searchNeighbors(vector.asInstanceOf[FlinkVector], rad)
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166211
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63706266
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    +        } else {
    +          0
    +        }
    +      }.sum
    +    }
    +
    +    /** Makes children nodes by partitioning the box into equal sub-boxes
    +      * and adding a node for each sub-box
    +      */
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5 * width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(
    +      center: Vector,
    +      width: Vector): Seq[Vector] = {
    +      def partitionHelper(
    +        box: Seq[Vector],
    +        dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up, down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node(((minVec.asBreeze + maxVec.asBreeze) * 0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +  /**
    +   * simple printing of tree for testing/debugging
    +   */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node) {
    +    if (node.children != null) {
    +      for (c <- node.children) {
    +        printTreeRecur(c)
    +      }
    +    } else {
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector) {
    +    insertRecur(queryPoint, root)
    +  }
    +
    +  private def insertRecur(
    +    queryPoint: Vector,
    +    node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox) {
    +        node.nodeElements += queryPoint
    +      } else {
    +        node.makeChildren()
    +        for (o <- node.nodeElements) {
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else {
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint a test point for which the method finds the minimal bounding
    +   *                   box that queryPoint lies in and returns elements in that boxes
    +   *                   siblings' leaf nodes
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    val ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            minNodes(queryPoint, c, nodeQueue)
    +          }
    +        } else {
    +          searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue PriorityQueue that stores all points in minimal bounding box of queryPoint
    +   */
    +  private def minNodes(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null) {
    +      nodeQueue += ((-node.minDist(queryPoint), node))
    +    } else {
    +      for (c <- node.children) {
    +        minNodes(queryPoint, c, nodeQueue)
    +      }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of queryPoint of a specified radius
    --- End diff --
    
    typo `neigiborhood`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-207685012
  
    @hsaputra I added apache/flink as upstream, namely:
        `git remote add upstream https://github.com/apache/flink.git`
    Then I ran what Chiwan above suggested, namely:
    ```
    # fetch updated master branch
    git fetch upstream master
    # checkout local master branch
    git checkout master 
    # merge local master branch and upstream master branch (this should be fast-forward merge.)
    git merge upstream/master
    # checkout local FLINK-1745 branch
    git checkout FLINK-1745
    # rebase FLINK-1745 on local master branch
    git rebase master
    # force push local FLINK-1745 branch to github's FLINK-1745 branch
    git push origin +FLINK-1745
    ```
    I then moved the 4 knn files originally in flink-staging/ to flink-libraries/ and pushed again. 
    
    The unfortunate thing now is that when I run `mvn clean package -DskipTests` I get errors (I can show you if you'd like....but I assume the Travic CI build won't go through and the error will pop up there too).  Did I do something wrong?  The good news is that I made a copy of the directory that I was working in since I've had rebasing problems before, so I can always try to go back to that and do a force push.
    
    I wonder since I'm only adding new files whether it's even easier to just clone `apache/master`, run `mvn clean package -DskipTests` put the new files in there and submit a new PR?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086835
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    +    val MaxArr = List.range(0, training.head.size).toArray
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    var trainingQuadTree = new QuadTree(MinVec, MaxVec, metric)
    +
    +    if (trainingQuadTree.maxPerBox < k) { /// make sure at least k points/box
    +      trainingQuadTree.maxPerBox = k
    +    }
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for (a <- testing) {
    +      /////  Find siblings' objects and do local kNN there
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166187
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086743
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    +    val MaxArr = List.range(0, training.head.size).toArray
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    var trainingQuadTree = new QuadTree(MinVec, MaxVec, metric)
    +
    +    if (trainingQuadTree.maxPerBox < k) { /// make sure at least k points/box
    +      trainingQuadTree.maxPerBox = k
    --- End diff --
    
    Done -- now in the constructor and now pass in `val maxPerBox = Array(k,20).max` into the constructor in KNN.scala


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093270
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    +          }
    +        }
    +        else {
    --- End diff --
    
    Please merge this line and above line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166181
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715078
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    --- End diff --
    
    why is `objects` a `var` if it is a mutable collection?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45618862
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    --- End diff --
    
    whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153081706
  
    Currently, although `useQuadTree` is set to false, the current logic creates unnecessary bounding boxes and quad tree object. This causes performance decreasing. So I think we need to refactor this logic. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153064094
  
    @tillrohrmann BTW, I have fixed that issue with the center, realized I can just pass a `copy` of `center` to `PartitionBox`.  Both the `KNNITSuite` and `QuadTreeSuite` are passing now.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166215
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-184847163
  
    Hi @chiwanpark sorry for the delay!  I will now have more time to wrap this PR up.  I added a check just before calling `knn`:
    ```scala
                      if (useQuadTree) {
                        if (metric.isInstanceOf[EuclideanDistanceMetric] ||
                          metric.isInstanceOf[SquaredEuclideanDistanceMetric]){
                          knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
                        } else {
                          throw new IllegalArgumentException(s" Error: metric must be" +
                            s" Euclidean or SquaredEuclidean!")
                        }
                      } else {
                        knnQueryBasic(training.values, testing.values, k, metric, queue, out)
                      }
                    }
                  }
                }
    ```
    Does that work?  The commit includes the hint for the cross operation as well. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166739
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47167755
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153078409
  
    Hi @danielblazevski, Thanks for updating your pull request. I'll review this soon. From short review, I think that we have to split predict method into two methods because lots of `if(useQuadTree)` statements are duplicated.
    
    I suggest creating two methods into `PredicateDataSetOperation`. One is finding kNN with quad-tree and the other is not. We can pass testing data set and training data set to the methods and receive result of the methods. I mean the logic in L191-L270 of `KNN.scala`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397472
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,146 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    +$$
    +
    +The brute-force approach is to compute the distance between every training and testing point.  To ease the brute-force computation of computing the distance between every traning point a quadtree is used.  The quadtree scales well in the number of training points, though poorly in the spatial dimension.  The algorithm will automatically choose whether or not to use the quadtree, though the user can override that decision by setting a parameter to force use or not use a quadtree. 
    +
    +##Operations
    +
    +`KNN` is a `Predictor`. 
    +As such, it supports the `fit` and `predict` operation.
    +
    +### Fit
    +
    +KNN is trained given a set of `LabeledVector`:
    +
    +* `fit: DataSet[LabeledVector] => Unit`
    +
    +### Predict
    +
    +KNN predicts for all subtypes of FlinkML's `Vector` the corresponding class label:
    +
    +* `predict[T <: Vector]: DataSet[T] => DataSet[(T, Array[Vector])]`, where the `(T, Array[Vector])` tuple
    +  corresponds to (testPoint, K-nearest training points)
    +
    +## Paremeters
    +The KNN implementation can be controlled by the following parameters:
    +
    +   <table class="table table-bordered">
    +    <thead>
    +      <tr>
    +        <th class="text-left" style="width: 20%">Parameters</th>
    +        <th class="text-center">Description</th>
    +      </tr>
    +    </thead>
    +
    +    <tbody>
    +      <tr>
    +        <td><strong>K</strong></td>
    +        <td>
    +          <p>
    +            Defines the number of nearest-neoghbors to search for.  That is, for each test point, the algorithm finds the K nearest neighbors in the training set
    +            (Default value: <strong>5</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong> DistanceMetric</strong></td>
    +        <td>
    +          <p>
    +            Sets the distance metric we use to calculate the distance between two points. If no metric is specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +            (Default value: <strong> EuclideanDistanceMetric </strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>Blocks</strong></td>
    +        <td>
    +          <p>
    +            Sets the number of blocks into which the input data will be split. This number should be set
    +            at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +            input [[DataSet]] is used as the number of blocks.
    +            (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>UseQuadTreeParam</strong></td>
    +        <td>
    +          <p>
    +             A boolean variable that whether or not to use a Quadtree to partition the training set to potentially simplify the KNN search.  If no value is specified, the code will automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well with the number of training and testing points, though poorly with the dimension.
    +            (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>SizeHint</strong></td>
    +        <td>
    +          <p>Specifies whether the training set or test set is small to optimize the cross product operation needed for the KNN search.  If the training set is small this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL` if the test set is small.
    +             (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +    </tbody>
    +  </table>
    +
    +## Examples
    +
    +{% highlight scala %}
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.classification.Classification
    +import org.apache.flink.ml.math.DenseVector
    +import org.apache.flink.ml.metrics.distances.
    +SquaredEuclideanDistanceMetric
    --- End diff --
    
    Could you move this line to end of previous line?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-160369019
  
    Only one of five Travis CI builds failed, the error message was 
    
    `[INFO] flink-connector-kafka .............................. FAILURE [01:48 min]`
    
    Strange that this only happened for one build, and not quite sure how to fix given that this seems to not have anything to do with the files I've added (I've only added new files, not changed existing ones, and did not use Kafka in any way)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63697916
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,145 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    --- End diff --
    
    `k` missing in first `KNN(b, A, k)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63732878
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    --- End diff --
    
    Shouldn't it be greater or equal?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153080192
  
    I'm not as keen on creating two methods @chiwanpark since there will be substantial overlap of code-- if we want to make a change in the future, we'd have to change two methods, and if we have to debug, we have to debug two methods.  And 3 `if(useQuadTree)` statements is not so bad, two of which are less than 5 lines.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r43646670
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,297 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Breeze,Vector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[Vector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    parameters.add(useQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object useQuadTreeParam extends Parameter[Boolean] {
    --- End diff --
    
    Object name should start with upper case letter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41491963
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    --- End diff --
    
    This is not very scalaesque. Better to do `val childrenNodes = cPart.map(p => new Node(p, mappedL, null))` with `val mappedL = L.map(x => x/2.0)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093216
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    --- End diff --
    
    Please re-align comment indentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714979
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    --- End diff --
    
    whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45620053
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    --- End diff --
    
    Why is `training` limited to a `DenseVector`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski closed the pull request at:

    https://github.com/apache/flink/pull/1220


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092558
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    --- End diff --
    
    To avoid using `var`, we can rewrite this method.
    
    ```scala
    def whichChild(queryPoint: Vector): Int = {
      (0 until queryPoint.size).map { i =>
        if (queryPoint(i) > center(i)) {
          Math.pow(2, queryPoint.size - 1 - i).toInt
        } else {
          0
        }
      }.sum
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-159314185
  
    @tillrohrmann thanks for the comments, glad it's looking like it's in good shape.  I'll make these changes soon in the next few days.  And thanks for the opportunity to contribute, have also begun the approximate kNN, have almost finished a Scala non-parallelized z-kNN for the approximate version of kNN.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-148112946
  
    I see @tillrohrmann.   The test was running without ending, and I assumed it had to do with the warning.  Realized that the partitionBox implementation that you gave is running in an infinite loop (this is the first time I got it to run).  I will debug or maybe try to incorporate some of the scalaesque ideas (nested functions, etc) into my implementation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63702930
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric, queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](
    +    training: Vector[T],
    +    testing: Vector[(Long, T)],
    +    k: Int, metric: DistanceMetric,
    +    queue: mutable.PriorityQueue[(FlinkVector,
    +      FlinkVector, Long, Double)],
    +    out: Collector[(FlinkVector,
    +      FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr = Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).max + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).max + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    --- End diff --
    
    I think it is more efficient to not create an array for a comparison. `math.max` and `math.min` should do the job.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086806
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    --- End diff --
    
    Done -- switched to val
    Done -- switched to `nodeElements`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086813
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166230
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue
    +   */
    +  private def MinNodes(queryPoint: Vector, node: Node, nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null){
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r50647908
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.DataSetUtils._
    +//import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint).get
    --- End diff --
    
    Nice, that fixed it ! I'll wait until I hear from you about the other parts before pushing here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-163443011
  
    I still have the old laptop, perhaps the best solution is, when I get a chance, to take the 2 files I modified back in that repo, kill this last commit and push again.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166315
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue
    +   */
    +  private def MinNodes(queryPoint: Vector, node: Node, nodeQueue: PriorityQueue[(Double, Node)]) {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145351367
  
    Ah, thanks that helps a lot.
    
    When I try to import `org.apache.flink.api.scala.utils._`, IntelliJ does not recognize it, probably this is because I started a few weeks ago.  Would you recommend forking a more recent branch of Flink, then try to do the incremental build by adding KNN.scala, QuadTree.scala  and a couple tests?  And if so, which should I just fork the most current master branch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45716124
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    +            MinNodes(obj,c,NodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(obj, child, NodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  private def MinNodes(obj:Vector,n:Node, NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if (n.children == null){
    +      NodeQueue += ((-n.minDist(obj), n))
    +    } else{
    +      for (c <- n.children) {
    +          MinNodes(obj,c, NodeQueue)
    +        }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of obj of a specified radius
    +    * scope is modified from original 2D version in:
    +    * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    +    *
    +    * original version only looks in minimal box; for the KNN Query, we look at
    +    * all nearby boxes. The radius is determined from searchNeighborsSiblingQueue
    +    * by defining a min-heap on the leaf nodes
    +   *
    +   * @param obj
    +   * @param radius
    +   * @return
    +   */
    +  def searchNeighbors(obj:Vector,radius:Double):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    --- End diff --
    
    why `var`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45620739
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    +    val MaxArr = List.range(0, training.head.size).toArray
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    var trainingQuadTree = new QuadTree(MinVec, MaxVec, metric)
    +
    +    if (trainingQuadTree.maxPerBox < k) { /// make sure at least k points/box
    +      trainingQuadTree.maxPerBox = k
    +    }
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for (a <- testing) {
    +      /////  Find siblings' objects and do local kNN there
    --- End diff --
    
    comment style


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092369
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    --- End diff --
    
    If the user set `UseQuadTreeParam` to true and use `CosineDistanceMetric` as distance metric, the value of `useQuadTree` will be true. It can occur unexpected error.
    
    What is exact meaning of `UseQuadTreeParam`? Does it mean force-use of quad tree?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47167003
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-148132199
  
    Ah... @tillrohrmann, I need to adjust my `whichChildren` method -- used in `insertRecur` --  to accommodate for your partitionBox, that may just well fix the infinite loop. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086809
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086847
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093744
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    --- End diff --
    
    Please remove a blank line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086750
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    --- End diff --
    
    Done (cool trick)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63712033
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    --- End diff --
    
    changed to to `scala.math`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63698367
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,145 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    +$$
    +
    +The brute-force approach is to compute the distance between every training and testing point.  To ease the brute-force computation of computing the distance between every traning point a quadtree is used.  The quadtree scales well in the number of training points, though poorly in the spatial dimension.  The algorithm will automatically choose whether or not to use the quadtree, though the user can override that decision by setting a parameter to force use or not use a quadtree. 
    +
    +##Operations
    +
    +`KNN` is a `Predictor`. 
    +As such, it supports the `fit` and `predict` operation.
    +
    +### Fit
    +
    +KNN is trained given a set of `LabeledVector`:
    +
    +* `fit: DataSet[LabeledVector] => Unit`
    +
    +### Predict
    +
    +KNN predicts for all subtypes of FlinkML's `Vector` the corresponding class label:
    +
    +* `predict[T <: Vector]: DataSet[T] => DataSet[(T, Array[Vector])]`, where the `(T, Array[Vector])` tuple
    +  corresponds to (testPoint, K-nearest training points)
    +
    +## Paremeters
    +The KNN implementation can be controlled by the following parameters:
    +
    +   <table class="table table-bordered">
    +    <thead>
    +      <tr>
    +        <th class="text-left" style="width: 20%">Parameters</th>
    +        <th class="text-center">Description</th>
    +      </tr>
    +    </thead>
    +
    +    <tbody>
    +      <tr>
    +        <td><strong>K</strong></td>
    +        <td>
    +          <p>
    +            Defines the number of nearest-neighbors to search for.  That is, for each test point, the algorithm finds the K-nearest neighbors in the training set
    +            (Default value: <strong>5</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>DistanceMetric</strong></td>
    +        <td>
    +          <p>
    +            Sets the distance metric we use to calculate the distance between two points. If no metric is specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +            (Default value: <strong>EuclideanDistanceMetric</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>Blocks</strong></td>
    +        <td>
    +          <p>
    +            Sets the number of blocks into which the input data will be split. This number should be set
    +            at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +            input [[DataSet]] is used as the number of blocks.
    +            (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>UseQuadTreeParam</strong></td>
    +        <td>
    +          <p>
    +             A boolean variable that whether or not to use a Quadtree to partition the training set to potentially simplify the KNN search.  If no value is specified, the code will automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well with the number of training and testing points, though poorly with the dimension.
    +            (Default value: <strong>None</strong>)
    --- End diff --
    
    Sorry my bad. Didn't read properly your description.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45620925
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    +    val MaxArr = List.range(0, training.head.size).toArray
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    var trainingQuadTree = new QuadTree(MinVec, MaxVec, metric)
    +
    +    if (trainingQuadTree.maxPerBox < k) { /// make sure at least k points/box
    +      trainingQuadTree.maxPerBox = k
    +    }
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for (a <- testing) {
    +      /////  Find siblings' objects and do local kNN there
    +      val siblingObjects =
    +        trainingQuadTree.searchNeighborsSiblingQueue(
    +          a._2.asInstanceOf[FlinkVector])
    --- End diff --
    
    instead of `for (a <- testing)` you could write `for ((id, vector) <- testing)` and then use the variables `id` and `vector`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63712405
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    +        } else {
    +          0
    +        }
    +      }.sum
    +    }
    +
    +    /** Makes children nodes by partitioning the box into equal sub-boxes
    +      * and adding a node for each sub-box
    +      */
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5 * width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(
    +      center: Vector,
    +      width: Vector): Seq[Vector] = {
    +      def partitionHelper(
    +        box: Seq[Vector],
    +        dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up, down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node(((minVec.asBreeze + maxVec.asBreeze) * 0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +  /**
    +   * simple printing of tree for testing/debugging
    +   */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node) {
    +    if (node.children != null) {
    +      for (c <- node.children) {
    +        printTreeRecur(c)
    +      }
    +    } else {
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector) {
    +    insertRecur(queryPoint, root)
    +  }
    +
    +  private def insertRecur(
    +    queryPoint: Vector,
    +    node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox) {
    +        node.nodeElements += queryPoint
    +      } else {
    +        node.makeChildren()
    +        for (o <- node.nodeElements) {
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else {
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint a test point for which the method finds the minimal bounding
    +   *                   box that queryPoint lies in and returns elements in that boxes
    +   *                   siblings' leaf nodes
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    val ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            minNodes(queryPoint, c, nodeQueue)
    +          }
    +        } else {
    +          searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue PriorityQueue that stores all points in minimal bounding box of queryPoint
    +   */
    +  private def minNodes(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null) {
    +      nodeQueue += ((-node.minDist(queryPoint), node))
    --- End diff --
    
    This is to make a min-heap.  I take all the siblings' leaf nodes of the node that `queryPoint` lies in, and of those nodes I want to extract the ones closest to `queryPoint`.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63710402
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric, queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](
    +    training: Vector[T],
    +    testing: Vector[(Long, T)],
    +    k: Int, metric: DistanceMetric,
    --- End diff --
    
    done, ran `cmd + alt + shift + L` in IntelliJ


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #1220: [FLINK-1745] Add exact k-nearest-neighbours algorithm to ...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the issue:

    https://github.com/apache/flink/pull/1220
  
    Hi @thvasilo, I've checked the [FlinkML index page](https://ci.apache.org/projects/flink/flink-docs-master/apis/batch/libs/ml/index.html#supervised-learning). There is a link to k-NN page in the index page. We find the link in the page for Flink 1.1 only because k-NN is merged to only master branch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-219258096
  
    @chiwanpark thanks!  Putting the finishing touches on [approximate](https://github.com/danielblazevski/flink/blob/FLINK-1934/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/zknn.scala) z-knn -- but also need to do a LSH version of approximate knn since z-knn only applies for dim < 30 (and z-value method is much quicker than LSH method for dim < 30, but at least LSH makes since for dim > 30).  Pretty excited about the performance gain compared to exact knn.    
    
    I'll be presenting on knn for Flink at a Scala meetup in NY at Spotify on May 24th and will definitely mention @chiwanpark and @tillrohrmann for all their help!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63711829
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    --- End diff --
    
    `minDist` has the key property:  `minDist(queryPoint, node)` represents the distance between a point and a box, and has the property that for any point `P` in the box `Dist(queryPoint, P) < minDist(queryPoint, node)`. 
    
     I only know -- off the top of my head -- how to define such a function with that key property for the Euclidean/SquareEuclidean metric.  Not 100% sure how to generalize this for arbitrary metrics -- could very well be possible, but would have to think more about it.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714597
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    --- End diff --
    
    Also methods with `Unit` return type should explicitly state it: `def printTree(): Unit = {...}`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-174329818
  
    @chiwanpark I see, I thought maybe there was a way to not even use a cross at all.  I changed the code according to your suggestion and got an error.  
    
    First, I assumed to add a line 
    ```scala
    val sizeHint = resultParameters.get(SizeHint).get
    ```
    before the 
    ```scala 
    val crossTuned = sizeHint match {...
    ``` 
    clause.  Attached is a screenshot form IntelliJ.  
    <img width="1280" alt="screenshot 2016-01-24 13 27 00" src="https://cloud.githubusercontent.com/assets/10012612/12538089/3d9801a4-c29e-11e5-9c8d-419c06fa7553.png">
    
    Another logistical question for @chiwanpark and @tillrohrmann is that I see the directory structure of Flink has changed since my initial PR.  I'm not sure what is the best practice here.  I see a couple of less-than-ideal options:  (1) create a new PR with updated directory structure, not ideal (2) pull the master branch, merge with this branch, but then when I commit many many commits will be added not relevant to this PR when I merge (less ideal...).  
    
    On a smaller note, I see your point @chiwanpark about raising the flag earlier with the choice of metric and using a quadtree.  Do we want to do this in `fit` though?  In `fit`, I can get the metric and the parameter `useQuadTree`, but if the user does not specify `setUseQuadTree`, then I still have a conservative test that requires one to know how many training and test points there are.  That will determine whether or not to use the quadtree (i.e. will only use a quadtree if it will improve performance based on a conservative test).  Is it OK to put in `predictValues` instead where all the variables needed -- metric, training  and test sets -- have been passed?  Otherwise I will have to re-factor the code more.  
    
    I changed the format based on @chiwanpark 's suggestion to make it look like what @tillrohrmann suggested.  
    
    I committed and pushed the code if you'd like (added a knn.md file in docs, but that is still very much a work in progress :-) 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166291
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714376
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    --- End diff --
    
    `else` should start in the same line as the closing curly braces of the if branch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46121896
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    --- End diff --
    
    Parameter description missing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086850
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    +            MinNodes(obj,c,NodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(obj, child, NodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  private def MinNodes(obj:Vector,n:Node, NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if (n.children == null){
    +      NodeQueue += ((-n.minDist(obj), n))
    +    } else{
    +      for (c <- n.children) {
    +          MinNodes(obj,c, NodeQueue)
    +        }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of obj of a specified radius
    +    * scope is modified from original 2D version in:
    +    * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    +    *
    +    * original version only looks in minimal box; for the KNN Query, we look at
    +    * all nearby boxes. The radius is determined from searchNeighborsSiblingQueue
    +    * by defining a min-heap on the leaf nodes
    +   *
    +   * @param obj
    +   * @param radius
    +   * @return
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715996
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    +            MinNodes(obj,c,NodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(obj, child, NodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  private def MinNodes(obj:Vector,n:Node, NodeQueue:PriorityQueue[(Double, Node)]) {
    --- End diff --
    
    Scaladocs


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086817
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086810
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-163468164
  
    In fact I think this was a bad mistake, and the best option I see is to close this PR and open a new one....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63707341
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    +        } else {
    +          0
    +        }
    +      }.sum
    +    }
    +
    +    /** Makes children nodes by partitioning the box into equal sub-boxes
    +      * and adding a node for each sub-box
    +      */
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5 * width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(
    +      center: Vector,
    +      width: Vector): Seq[Vector] = {
    +      def partitionHelper(
    +        box: Seq[Vector],
    +        dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up, down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node(((minVec.asBreeze + maxVec.asBreeze) * 0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +  /**
    +   * simple printing of tree for testing/debugging
    +   */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node) {
    +    if (node.children != null) {
    +      for (c <- node.children) {
    +        printTreeRecur(c)
    +      }
    +    } else {
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector) {
    +    insertRecur(queryPoint, root)
    +  }
    +
    +  private def insertRecur(
    +    queryPoint: Vector,
    +    node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox) {
    +        node.nodeElements += queryPoint
    +      } else {
    +        node.makeChildren()
    +        for (o <- node.nodeElements) {
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else {
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint a test point for which the method finds the minimal bounding
    +   *                   box that queryPoint lies in and returns elements in that boxes
    +   *                   siblings' leaf nodes
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    val ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            minNodes(queryPoint, c, nodeQueue)
    +          }
    +        } else {
    +          searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue PriorityQueue that stores all points in minimal bounding box of queryPoint
    +   */
    +  private def minNodes(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null) {
    +      nodeQueue += ((-node.minDist(queryPoint), node))
    --- End diff --
    
    why the negative `minDist` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-220047418
  
    Thanks @tillrohrmann, made changes as per your suggestions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086846
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    --- End diff --
    
    done -- changed to queryPoint


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086751
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    --- End diff --
    
    Done -- removed casting


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397260
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,146 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    +$$
    +
    +The brute-force approach is to compute the distance between every training and testing point.  To ease the brute-force computation of computing the distance between every traning point a quadtree is used.  The quadtree scales well in the number of training points, though poorly in the spatial dimension.  The algorithm will automatically choose whether or not to use the quadtree, though the user can override that decision by setting a parameter to force use or not use a quadtree. 
    +
    +##Operations
    +
    +`KNN` is a `Predictor`. 
    +As such, it supports the `fit` and `predict` operation.
    +
    +### Fit
    +
    +KNN is trained given a set of `LabeledVector`:
    +
    +* `fit: DataSet[LabeledVector] => Unit`
    +
    +### Predict
    +
    +KNN predicts for all subtypes of FlinkML's `Vector` the corresponding class label:
    +
    +* `predict[T <: Vector]: DataSet[T] => DataSet[(T, Array[Vector])]`, where the `(T, Array[Vector])` tuple
    +  corresponds to (testPoint, K-nearest training points)
    +
    +## Paremeters
    +The KNN implementation can be controlled by the following parameters:
    +
    +   <table class="table table-bordered">
    +    <thead>
    +      <tr>
    +        <th class="text-left" style="width: 20%">Parameters</th>
    +        <th class="text-center">Description</th>
    +      </tr>
    +    </thead>
    +
    +    <tbody>
    +      <tr>
    +        <td><strong>K</strong></td>
    +        <td>
    +          <p>
    +            Defines the number of nearest-neoghbors to search for.  That is, for each test point, the algorithm finds the K nearest neighbors in the training set
    +            (Default value: <strong>5</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong> DistanceMetric</strong></td>
    --- End diff --
    
    Please remove space before DistanceMetric


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-151942911
  
    @tillrohrmann I now have more time to go back and try to finalize this PR in the next couple of weeks.  After debugging a bit, I noticed that in your modification of `partitionBox`, the variable `center` is different before and after the call of `partitionBox` in `makeChildren`.  For example, in `makeChildren` I added some lines to print to the console, namely
    ``` scala
         println("center before partitioning =  " + center)
          val cPart = partitionBox(center, width)
          println("cPart =  " + cPart)
          val mappedWidth = 0.5*width.asBreeze
          children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
          println("center after partitioning =  " + center)
    ```
    The output to console is
    ```
    center before partitioning =  DenseVector(0.0, 0.0)
    cPart =  List(DenseVector(-0.5, -0.25), DenseVector(-0.5, 0.25), DenseVector(0.5, -0.25), DenseVector(0.5, 0.25))
    center after partitioning =  DenseVector(0.5, 0.25)
    ```
    So the output `cPart` looks good, but the value of `center` after partitioning should still be `(0.0,0.0)`.  I'm confused as to how it is even changed to `(0.5, 0.25)` the final entry of `cPart`, and hence not clear how to fix that.  I imagine it should be an easy fix; of course I can use a hack to update `center` to be the average of `cPart`, but that seems wasteful since `center`for a given node should not be changed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46123721
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    --- End diff --
    
    Concerning my last comment I thought about something like
    
    ```
    def searchSiblingQueue(
        queryPoint: Vector,
        node: Node,
        nodeQueue: PriorityQueue[(Double, Node)]): Unit = {
      if (node.children == null) {
      	MinNodes(queryPoint, node, nodeQueue)
      } else {
        for (child <- node.children; if child.contains(queryPoint)) {
          searchSiblingQueue(queryPoint, child, nodeQueue)
        }
      }
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086833
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    +    val MaxArr = List.range(0, training.head.size).toArray
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    var trainingQuadTree = new QuadTree(MinVec, MaxVec, metric)
    +
    +    if (trainingQuadTree.maxPerBox < k) { /// make sure at least k points/box
    +      trainingQuadTree.maxPerBox = k
    +    }
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for (a <- testing) {
    +      /////  Find siblings' objects and do local kNN there
    +      val siblingObjects =
    +        trainingQuadTree.searchNeighborsSiblingQueue(
    +          a._2.asInstanceOf[FlinkVector])
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166264
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41525174
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    --- End diff --
    
    @tillrohrmann, I had an issue implementing this using Flink's Vector API.  Changing `L` to `width`,  `val mappedWidth = width.map(x => x/2.0)` results in an error.  I searched a bit for this, and a hack I tried `val mappedWidth = width.map(x => x.asInstance[Double]/2.0)`, but that seems to result in a iterable, which cannot be plugged into `Node(center:Vector,width:Vector, var children:ListBuffer[Node])` when constructing `childrenNodes`.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093740
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    --- End diff --
    
    Please remove a blank line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086811
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093251
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    --- End diff --
    
    Need a space after `else`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093265
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    --- End diff --
    
    We can use `val` in this case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46121953
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    --- End diff --
    
    Parameter list formatting not consistent with rest of the code base.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45715406
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    --- End diff --
    
    Formatting of parameter list


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61398228
  
    --- Diff: flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/classification/Classification.scala ---
    @@ -131,3 +131,6 @@ object Classification {
     
       val expectedWeightVector = DenseVector(-1.95, -3.45)
     }
    +
    +
    +
    --- End diff --
    
    Are these new lines necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
GitHub user danielblazevski reopened a pull request:

    https://github.com/apache/flink/pull/1220

    [FLINK-1745] Add exact k-nearest-neighbours algorithm to machine learning library

    I added a quadtree data structure for the knn algorithm.  @chiwanpark made originally made a pull request for a kNN algorithm, and we coordinated so that I incorporate a tree structure. The quadtree scales very well with the number of training + test points, but scales poorly with the dimension (even the R-tree scales poorly with the dimension). I added a flag that is automatically determines whether or not to use the quadtree. My implementation needed to use the Euclidean or SquaredEuclidean distance since I needed a specific notion of the distance between a test point and a box in the quadtree. I added another test KNNQuadTreeSuite in addition to Chiwan Park's KNNITSuite, since C. Park's parameters will automatically choose the brute-force non-quadtree method.
    
    For more details on the quadtree + how I used it for the KNN query, please see another branch I created that has a README.md:
    https://github.com/danielblazevski/flink/tree/FLINK-1745-devel/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/danielblazevski/flink FLINK-1745

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1220.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1220
    
----
commit c7e5056c6d273f6f0f841f77e0fdd91ca221602d
Author: Chiwan Park <ch...@apache.org>
Date:   2015-06-30T08:41:25Z

    [FLINK-1745] [ml] Add exact k-nearest-neighbor join

commit 9d0c7942c09086324fadb29bdce749683a0d1a7e
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-15T21:49:05Z

    modified kNN test to familiarize with Flink and KNN.scala

commit 611248e57166dc549f86f805b590dd4e45cb3df5
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-15T21:49:17Z

    modified kNN test to familiarize with Flink and KNN.scala

commit 1fd8231ce194b52b5a1bd55bbc5e135b3fa5775b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-16T01:26:57Z

    nightly commit, minor changes:  got the filter to work, working on mapping the training set to include box lables

commit 15d7d2cb308b23e24c43d103b85a76b0e665cbd3
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-22T02:02:51Z

    commit before incporporating quadtree

commit 8f2da8a66516565c59df8828de2715b45397cb7f
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-22T15:49:25Z

    did a basic import of QuadTree and Test; to-do:  modify QuadTree to allow KNN.scala to make use of

commit e1cef2c5aea65c6f204caeff6348e2778231f98d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-22T21:03:04Z

    transfered ListBuffers for objects in leaf nodes to Vectors

commit c3387ef2ef59734727b56ea652fdb29af957d20b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T00:41:29Z

    basic test on 2D unit box seems to work -- need to generalize, e.g. to include automated bounding box

commit 48294ff37a5f800e5111280da5a3c03f4375028d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T15:03:06Z

    had to debug quadtree -- back to testing 2D

commit 6403ba14e240ed8d67a296ac789e7e00dece800d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T15:22:46Z

    Testing 2D looks good, strong improvement in run time compared to brute-force method

commit 426466a40bc2625f390fe0d912f56a346e46c8f8
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-23T19:04:52Z

    added automated detection of bounding box based on min/max values of both training and test sets

commit c35543b828384aa4ce04d56dfcb3d73db46d1e6d
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T00:28:56Z

    added automated radius about test point to define localized neighborhood, result runs.  TO-DO:  Lots of tests

commit 8e2d2e78f8533d4192aebe9b4baa7efbfa5928a5
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T00:54:06Z

    Note for future:  previous commit passed test of Chiwan Park had in intial knn implementation

commit d6fd40cb88d6e198e52c368e829bf7d32d432081
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T01:56:38Z

    Note for future:  previous commit passed 3D version of the test that Chiwan Park had in the intial knn implementation

commit 0ec1f4866157ca073341672e7fe9a50871ac0b7c
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T14:27:20Z

    changed filename of QuadTreeTest to QuadTreeSuite, about to make test more comprehensive and similar format to other Flink tests

commit ac81561cad27b65d158ae08fd0fb15bdb51d1c8b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T19:51:32Z

    refactored testing of QuadTree, and added more tests

commit b17f82d5ce0214617c8dbc4a387410057d6f3832
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T22:49:10Z

    added KNNBenchmark to check runtimes

commit 530565835d4b5934fcac9e0e51105bb669fec9be
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-24T22:49:17Z

    added KNNBenchmark to check runtimes

commit 1f946cb30450604e92bbd0f5959ce9a60eb4c41b
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-25T01:13:00Z

    fixed bug -- in find siblings, needed to search for minimal bounding boxes

commit 22e4eb7b57795ad1ca4392ca1c1a8bdae76afa8e
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-27T03:34:16Z

    added more thorough benchmark files; about to modify  bounding box to only bound training set and modify search for the testing set

commit 3723f6b09ec7d45f6444df70a5f699cbf998a4bb
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-27T03:34:21Z

    added more thorough benchmark files; about to modify  bounding box to only bound training set and modify search for the testing set

commit c41d3e1029bf81a37cf3594f202b904e2d99e3ac
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-28T03:53:12Z

    major simplification in choosing the radius to look at nearby neighbors

commit 7c77ea20fd9e8a0c4a33c81b83187c84b380d6b2
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-28T04:03:59Z

    cleaned up; commit before deleting previous sibling search

commit cf4aa5d75611db19040466040c3d29432cb0e5f7
Author: danielblazevski <da...@gmail.com>
Date:   2015-09-28T14:02:51Z

    added new devel branch to push temp changes on github

commit ec6ddb0a57136075b4f77616e6e48eb5bcc50a11
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T00:17:33Z

    cleaned up; removed comments and a unused method

commit 7ed9926d8207b5f59b4ceb968d7ebd732029f5c3
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T20:28:31Z

    fixed bug in bFiltVect, and renamed to trainingFiltered

commit 4b3bb2ec92396bf754d0d207ffc6853406ce7c39
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T21:05:46Z

    fixed bug:  if there are fewer than maxPerBox total training points, do not do heap construction, just make siblingsQueue = root.objects

commit 1662b38822dfdfbbca272d377fa3b94f8246e9e6
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T22:03:42Z

    added metric to constructor, and added a flag to test whether it is Euclidean or SquaredEuclidean

commit f654b841cc8d91fa861f188469831404c288b227
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T22:48:51Z

    changed name of test that uses the Quadtree along with KNN -- modified from CHiwan Park's test to ensure flag to use Quadtree will pass

commit 7928798281dba5554eeb63df7e67400a42e7a381
Author: danielblazevski <da...@gmail.com>
Date:   2015-10-01T22:54:25Z

    fixed QuadTree test to conform to using a min-heap

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092532
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    --- End diff --
    
    Why we need a new exception type? I think `IllegalArgumentException` could be used in this case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146175315
  
    @chiwanpark, in lines 203-207
    +                  val useQuadTree = resultParameters.get(useQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length)/
    +                      math.log(4.0)) < math.log(training.values.length)/math.log(4.0) &&
    +                    (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                      metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    the code decides whether to use quadtree or not if no value is specified.  This codes decides based on the number of training + test points + dimension, and is a conservative estimate so that when it uses the quadtree, the quadtree will improve performance compared to the brute-force method -- basically the quadtree scales poorly with dimension, but really well with the number of points. 
    
    As for using a `Vector` for `minVec` and `maxVec`, I plug in `minVec` and `maxVec` to construct the root Node, and I found it best to use a ListBuffer in the constructor for the Node class when partitioning the boxes into sub-boxes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-209974474
  
    @chiwanpark I added those file, forgot to run a `git add` statement.  A couple of other files were added to the `flink-staging` directory, perhaps as a result of rebasing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-219702506
  
    Great to hear that z-knn is almost done! If you think the implementation has good shape, do not hesitate to open a pull request.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63703023
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values, testing.values, k, metric, queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric, queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](
    +    training: Vector[T],
    +    testing: Vector[(Long, T)],
    +    k: Int, metric: DistanceMetric,
    +    queue: mutable.PriorityQueue[(FlinkVector,
    +      FlinkVector, Long, Double)],
    +    out: Collector[(FlinkVector,
    +      FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr = Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).max + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).max + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k, 20).max
    --- End diff --
    
    `math.max(k, 20)` should be more efficient


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146175924
  
    @chiwanpark,  I could see if I can do a conversion from ListBuffer to Vector so that at least the constructor for the QuadTree is a Vector, which I agree is cleaner.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145365458
  
    Sounds good, unless I hear otherwise, I will wait until I hear back from you again before I move further with getting the Travis CI build to go through.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146208044
  
    These are good comments @chiwanpark.  I will change the use of `useQuadTree` and test your partitionBox when I get a chance in the next few days.  I am using IntelliJ


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166195
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41492287
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    --- End diff --
    
    parameter name could be a bit more descriptive. At least add some scalaDocs for this class with their parameters


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166199
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166190
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397153
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,146 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    +$$
    +
    +The brute-force approach is to compute the distance between every training and testing point.  To ease the brute-force computation of computing the distance between every traning point a quadtree is used.  The quadtree scales well in the number of training points, though poorly in the spatial dimension.  The algorithm will automatically choose whether or not to use the quadtree, though the user can override that decision by setting a parameter to force use or not use a quadtree. 
    +
    +##Operations
    +
    +`KNN` is a `Predictor`. 
    +As such, it supports the `fit` and `predict` operation.
    +
    +### Fit
    +
    +KNN is trained given a set of `LabeledVector`:
    +
    +* `fit: DataSet[LabeledVector] => Unit`
    +
    +### Predict
    +
    +KNN predicts for all subtypes of FlinkML's `Vector` the corresponding class label:
    +
    +* `predict[T <: Vector]: DataSet[T] => DataSet[(T, Array[Vector])]`, where the `(T, Array[Vector])` tuple
    +  corresponds to (testPoint, K-nearest training points)
    +
    +## Paremeters
    +The KNN implementation can be controlled by the following parameters:
    +
    +   <table class="table table-bordered">
    +    <thead>
    +      <tr>
    +        <th class="text-left" style="width: 20%">Parameters</th>
    +        <th class="text-center">Description</th>
    +      </tr>
    +    </thead>
    +
    +    <tbody>
    +      <tr>
    +        <td><strong>K</strong></td>
    +        <td>
    +          <p>
    +            Defines the number of nearest-neoghbors to search for.  That is, for each test point, the algorithm finds the K nearest neighbors in the training set
    --- End diff --
    
    Defines the number of nearest-_neighbors_ ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-222507122
  
    @chiwanpark the formatting did not work, see [screenshot](https://www.dropbox.com/s/psrercxcikozjgd/Screenshot%202016-05-30%2010.38.48.png?dl=0) in my community edition of IntelliJ.  According to the PR, there should be a "Import from IntelliJ IDEA code style XML" option.  I'm going to paste this same comment in the PR


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-222481628
  
    Thanks @chiwanpark !  I saw that you changed the formatting of the code.  Did you automatically do this in IntelliJ?  I've been using `cmd + alt + shift + L` in IntelliJ, and the formatting is a bit different (and not as nice).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145279351
  
    @danielblazevski Hi, Thanks for contribution. I'll review this soon. :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46121873
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    --- End diff --
    
    Parameter description


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63704597
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    --- End diff --
    
    `Math` is Java. Why not using `scala.math`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086746
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    --- End diff --
    
    Good point, added a .clone()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-200465211
  
    Hi Daniel, the docs are handwritten as far as I know.
    
    On Wed, Mar 23, 2016 at 5:49 PM, Daniel Blazevski <no...@github.com>
    wrote:
    
    > @chiwanpark <https://github.com/chiwanpark> about the docs, when I look
    > at docs/libs/ml/smv.md for instance (or als.md, etc.), the parameters
    > section seems auto-generated, is that correct? Is so, do you know how this
    > was auto-generated? If not, I'll use the existing docs as a template.
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/1220#issuecomment-200436234>
    >



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714274
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    --- End diff --
    
    opening curly braces start in same line as if condition


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086753
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    --- End diff --
    
    Done -- removed DenseVector


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-163438805
  
    @tillrohrmann and @chiwanpark made changes based on your comments.  Also note that I recenly got a new laptop, and I decided to install IntelliJ and download the most up-to-date version of Flink's source code and added the 4 relevant files (KNN.scala, QuadTree.scala and the 2 associated test suites) and work on that.  Did not realize that this would affect my PR, now the PR shows that 301 files have been changed :-(


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-170875879
  
    Hi @danielblazevski, I'm sorry for late reply. If you turn off IntelliJ IDEA align option (Turn off Preferences -> Editor -> Code Style -> Scala -> Wrapping and Braces -> Method declaration parameters -> Align when multiline), you can get style that is suggested by @tillrohrmann.
    
    Could you apply this option?
    
    I think that your PR is almost ready to merge. But I have to check few problems that still exist.
    
    First, about a meaning of `UseQuadTree` parameter, you said that it means force-use quadtree. I think this would be a problem because `DistanceMeasure` parameter can be conflict with quadtree. I would like to raise an error earlier if the parameter setting has a problem. Could you add this into top of fit operation?
    
    Second, how about avoiding `cross` operation? As @tillrohrmann said, `cross` operation is a very heavy operation. Is there any nicer solution to this?
    
    Other problems such as some difference styles, unnecessary spaces can be addressed by me before merge this. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by hsaputra <gi...@git.apache.org>.
Github user hsaputra commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-207547199
  
    @danielblazevski : Sorry, but could you help rebase the conflicts for this PR? Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086844
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    --- End diff --
    
    Done -- changed whitespaces, I assume
    Done -- changed to `nodeQueue`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153409659
  
    @chiwanpark , refactored the code using the first method, reads nice and very minimal duplicated code.  Also imported Scala Vectors and renamed Flink's Vectors to FlinkVector as per your email (thanks, btw!). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093852
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue
    +   */
    +  private def MinNodes(queryPoint: Vector, node: Node, nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null){
    +      nodeQueue += ((-node.minDist(queryPoint), node))
    +    } else{
    +      for (c <- node.children) {
    +          MinNodes(queryPoint, c, nodeQueue)
    +        }
    --- End diff --
    
    Wrong indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145508479
  
    Changed the title of the PR.
    
    On Mon, Oct 5, 2015 at 3:28 AM, Ufuk Celebi <no...@github.com>
    wrote:
    
    > Thanks for this impressive PR. A minor comment: could you edit the title
    > of the PR to include more details than the issue ID (we can't do it because
    > the ASF is managing the github account). It's hard to track the PRs
    > otherwise. I would just go with the JIRA title: [FLINK-1745] Add exact
    > k-nearest-neighbours algorithm to machine learning library
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/1220#issuecomment-145449146>.
    >



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093242
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    --- End diff --
    
    Please add a space after comma.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166159
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63704440
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    --- End diff --
    
    Why does it only work with `Euclidean` metric?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093728
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k,20).max
    +    val trainingQuadTree = new QuadTree(MinVec, MaxVec, metric, maxPerBox)
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for  ((id, vector) <- testing) {
    +      //  Find siblings' objects and do local kNN there
    +      val siblingObjects =
    +        trainingQuadTree.searchNeighborsSiblingQueue(
    +          vector.asInstanceOf[FlinkVector])
    +
    +      // do KNN query on siblingObjects and get max distance of kNN
    +      // then rad is good choice for a neighborhood to do a refined
    +      // local kNN search
    +      val knnSiblings = siblingObjects.map(v => metric.distance(vector, v)
    +      ).sortWith(_ < _).take(k)
    +
    +      val rad = knnSiblings.last
    +      var trainingFiltered = new ListBuffer[FlinkVector]
    +      trainingFiltered =
    +        trainingQuadTree.searchNeighbors(vector.asInstanceOf[FlinkVector], rad)
    --- End diff --
    
    Why we use `var` in this case?
    
    ```scala
    val trainingFiltered = trainingQuadTree.searchNeighbors(vector, rad)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47168128
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    --- End diff --
    
    Yes, it means force-use of quad-tree


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093714
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k,20).max
    +    val trainingQuadTree = new QuadTree(MinVec, MaxVec, metric, maxPerBox)
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    --- End diff --
    
    Because `T` is already derived by `FlinkVector`, I think we don't need casting `T` to `FlinkVector`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45713645
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    --- End diff --
    
    two spaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61398006
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    --- End diff --
    
    Could you add a scaladoc for this method? All public methods should have a scaladoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45712745
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    --- End diff --
    
    whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397754
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    --- End diff --
    
    We need to rename `UseQuadTree` parameter to `useQuadTree`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63856740
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    --- End diff --
    
    Maybe we could also add a reference to where this condition comes from.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45620538
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,321 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                    knnQueryWithQuadTree(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values.asInstanceOf[Vector[DenseVector]],
    +                      testing.values,k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[DenseVector],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = List.range(0, training.head.size).toArray
    --- End diff --
    
    You can use `Array.tabulate(training.head.size)(x => x)` to directly generate an Array with the sequence.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41992974
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    --- End diff --
    
    The reason for this was that we didn't want to reimplement a linear algebra library. And at the same time we didn't want the people to lock in on one specific library. So theoretically, you could also use a different math backend. You just have to convert it from `Vector` to the library specific type.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45716139
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    +            MinNodes(obj,c,NodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(obj, child, NodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  private def MinNodes(obj:Vector,n:Node, NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if (n.children == null){
    +      NodeQueue += ((-n.minDist(obj), n))
    +    } else{
    +      for (c <- n.children) {
    +          MinNodes(obj,c, NodeQueue)
    +        }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of obj of a specified radius
    +    * scope is modified from original 2D version in:
    +    * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    +    *
    +    * original version only looks in minimal box; for the KNN Query, we look at
    +    * all nearby boxes. The radius is determined from searchNeighborsSiblingQueue
    +    * by defining a min-heap on the leaf nodes
    +   *
    +   * @param obj
    +   * @param radius
    +   * @return
    --- End diff --
    
    parameter descriptions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63703823
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    --- End diff --
    
    this condition could written more succinctly via `(0 until queryPoint.size).forall{...}`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63702415
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    --- End diff --
    
    why do you create the `PriorityQueue` here and not in the `knnQueryBasic` and `knnQueryWithQuadTree` respectively?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46121884
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    --- End diff --
    
    Scaladocs missing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41491996
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    +      children = Childrennodes.clone()
    +    }
    +
    +    /**
    +      * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +      * plane through the center of the box keeping the n-th coordinate fixed,
    +      *  then shifting it in the n-th direction up and down
    +      * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param cPart
    +     * @param L
    +     * @param dim
    +     * @return
    --- End diff --
    
    Description of parameter would be helpful here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-171277446
  
    Hi @chiwanpark , I agree `cross` is an expensive computation.  That part of the code was adopted from your earlier version.  Before I try to change it, do you have ideas on the best strategy to fix it using the most efficient Flink-esque features?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166247
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue
    +   */
    +  private def MinNodes(queryPoint: Vector, node: Node, nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null){
    +      nodeQueue += ((-node.minDist(queryPoint), node))
    +    } else{
    +      for (c <- node.children) {
    +          MinNodes(queryPoint, c, nodeQueue)
    +        }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of queryPoint of a specified radius
    +    * scope is modified from original 2D version in:
    +    * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    +    *
    +    * original version only looks in minimal box; for the KNN Query, we look at
    +    * all nearby boxes. The radius is determined from searchNeighborsSiblingQueue
    +    * by defining a min-heap on the leaf nodes
    +   *
    +   * @param queryPoint
    +   * @param radius
    +   * @return all points within queryPoint with given radius
    +   */
    +  def searchNeighbors(queryPoint: Vector, radius: Double): ListBuffer[Vector] = {
    +    val ret = new ListBuffer[Vector]
    +    searchRecur(queryPoint, radius, root, ret)
    +    ret
    +  }
    +
    +  private def searchRecur(queryPoint: Vector, radius: Double, node: Node, ret: ListBuffer[Vector]) {
    +    if (node.children == null) {
    +      ret ++= node.nodeElements
    +    } else {
    +      for(child <- node.children; if child.isNear(queryPoint,radius)) {
    +        searchRecur(queryPoint, radius, child, ret)
    +      }
    +    }
    +  }
    +
    +   def distance(a: Vector, b: Vector):Double = {
    +     distMetric.distance(a, b)
    +  }
    --- End diff --
    
    done -- good catch!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166169
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166138
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166144
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63712157
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    /** Tests whether the queryPoint is in the node, or a child of that node
    +      *
    +      * @param queryPoint
    +      * @return
    +      */
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    +        } else {
    +          0
    +        }
    +      }.sum
    +    }
    +
    +    /** Makes children nodes by partitioning the box into equal sub-boxes
    +      * and adding a node for each sub-box
    +      */
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5 * width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(
    +      center: Vector,
    +      width: Vector): Seq[Vector] = {
    +      def partitionHelper(
    +        box: Seq[Vector],
    +        dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up, down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node(((minVec.asBreeze + maxVec.asBreeze) * 0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +  /**
    +   * simple printing of tree for testing/debugging
    +   */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node) {
    +    if (node.children != null) {
    +      for (c <- node.children) {
    +        printTreeRecur(c)
    +      }
    +    } else {
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector) {
    +    insertRecur(queryPoint, root)
    +  }
    +
    +  private def insertRecur(
    +    queryPoint: Vector,
    +    node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox) {
    +        node.nodeElements += queryPoint
    +      } else {
    +        node.makeChildren()
    +        for (o <- node.nodeElements) {
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else {
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint a test point for which the method finds the minimal bounding
    +   *                   box that queryPoint lies in and returns elements in that boxes
    +   *                   siblings' leaf nodes
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    val ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            minNodes(queryPoint, c, nodeQueue)
    +          }
    +        } else {
    +          searchRecurSiblingQueue(queryPoint, child, nodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Goes down to minimal bounding box of queryPoint, and add elements to nodeQueue
    +   *
    +   * @param queryPoint point under consideration
    +   * @param node node that queryPoint lies in
    +   * @param nodeQueue PriorityQueue that stores all points in minimal bounding box of queryPoint
    +   */
    +  private def minNodes(
    +    queryPoint: Vector,
    +    node: Node,
    +    nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children == null) {
    +      nodeQueue += ((-node.minDist(queryPoint), node))
    +    } else {
    +      for (c <- node.children) {
    +        minNodes(queryPoint, c, nodeQueue)
    +      }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of queryPoint of a specified radius
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714098
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    --- End diff --
    
    whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61398096
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(
    +  minVec: Vector,
    +  maxVec: Vector,
    +  distMetric: DistanceMetric,
    +  maxPerBox: Int) {
    +
    +  class Node(
    +    center: Vector,
    +    width: Vector,
    +    var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      val count = (0 until queryPoint.size).filter { i =>
    +        (queryPoint(i) - radius < center(i) + width(i) / 2) &&
    +          (queryPoint(i) + radius > center(i) - width(i) / 2)
    +      }.size
    +
    +      count == queryPoint.size
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(
    +      queryPoint: Vector,
    +      radius: Double): Boolean = {
    +      minDist(queryPoint) < radius
    +    }
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +    def minDist(queryPoint: Vector): Double = {
    +      val minDist = (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        } else {
    +          0
    +        }
    +      }.sum
    +
    +      distMetric match {
    +        case _: SquaredEuclideanDistanceMetric => minDist
    +        case _: EuclideanDistanceMetric => math.sqrt(minDist)
    +        case _ => throw new IllegalArgumentException(s" Error: metric must be" +
    +          s" Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      (0 until queryPoint.size).map { i =>
    +        if (queryPoint(i) > center(i)) {
    +          Math.pow(2, queryPoint.size - 1 - i).toInt
    +        } else {
    +          0
    +        }
    +      }.sum
    +    }
    +
    +    def makeChildren() {
    --- End diff --
    
    We can change scope of this method to private.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47168049
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086799
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    if (root.children == null) {   // edge case when the main box has not been partitioned at all
    +      root.objects
    +    } else {
    +      var NodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(subOne))
    +      searchRecurSiblingQueue(obj, root, NodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = NodeQueue.dequeue()
    +        if (dq._2.objects.nonEmpty) {
    +          ret ++= dq._2.objects
    +          count += dq._2.objects.length
    +        }
    +      }
    +      ret
    +    }
    +}
    +
    +  private def searchRecurSiblingQueue(obj:Vector,n:Node,
    +                                      NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if(n.children != null) {
    +      for(child <- n.children; if child.contains(obj)) {
    +        if (child.children == null) {
    +          for (c <- n.children) {
    +            ////// Go down to minimal bounding box
    +            MinNodes(obj,c,NodeQueue)
    +          }
    +        }
    +        else {
    +            searchRecurSiblingQueue(obj, child, NodeQueue)
    +        }
    +      }
    +    }
    +  }
    +
    +  private def MinNodes(obj:Vector,n:Node, NodeQueue:PriorityQueue[(Double, Node)]) {
    +    if (n.children == null){
    +      NodeQueue += ((-n.minDist(obj), n))
    +    } else{
    +      for (c <- n.children) {
    +          MinNodes(obj,c, NodeQueue)
    +        }
    +    }
    +  }
    +
    +  /** Finds all objects within a neigiborhood of obj of a specified radius
    +    * scope is modified from original 2D version in:
    +    * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    +    *
    +    * original version only looks in minimal box; for the KNN Query, we look at
    +    * all nearby boxes. The radius is determined from searchNeighborsSiblingQueue
    +    * by defining a min-heap on the leaf nodes
    +   *
    +   * @param obj
    +   * @param radius
    +   * @return
    +   */
    +  def searchNeighbors(obj:Vector,radius:Double):ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    --- End diff --
    
    done, now a val


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093243
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    --- End diff --
    
    Need a space after `else`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166275
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k,20).max
    +    val trainingQuadTree = new QuadTree(MinVec, MaxVec, metric, maxPerBox)
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for  ((id, vector) <- testing) {
    +      //  Find siblings' objects and do local kNN there
    +      val siblingObjects =
    +        trainingQuadTree.searchNeighborsSiblingQueue(
    +          vector.asInstanceOf[FlinkVector])
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r50647700
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.DataSetUtils._
    +//import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint).get
    --- End diff --
    
    This line should be `val sizeHint = resultParameters.get(SizeHint)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46123027
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    --- End diff --
    
    Why do you add all `children` if only one `child` contains the `queryPoint`? Do you want to store the neighbouring children in `queryPoint` as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-215384494
  
    @chiwanpark Thanks for the comments.  I made all the changes except making `makeChildren` private since that is in the `Node` class and is called in the `Quadtree` class outside of the `Node` class.  Since it's still public, I added Scaladocs as per your comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r61397574
  
    --- Diff: docs/libs/ml/knn.md ---
    @@ -0,0 +1,146 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - k-nearest neighbors
    +title: <a href="../ml">FlinkML</a> - knn
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Description
    +Implements an exact k-nearest neighbors algorithm.  Given a training set $A$ and a testing set $B$, the algorithm returns
    +
    +$$
    +KNN(A,B, k) = \{ \left( b, KNN(b,A) \right) where b \in B and KNN(b, A, k) are the k-nearest points to b in A \}
    +$$
    +
    +The brute-force approach is to compute the distance between every training and testing point.  To ease the brute-force computation of computing the distance between every traning point a quadtree is used.  The quadtree scales well in the number of training points, though poorly in the spatial dimension.  The algorithm will automatically choose whether or not to use the quadtree, though the user can override that decision by setting a parameter to force use or not use a quadtree. 
    +
    +##Operations
    +
    +`KNN` is a `Predictor`. 
    +As such, it supports the `fit` and `predict` operation.
    +
    +### Fit
    +
    +KNN is trained given a set of `LabeledVector`:
    +
    +* `fit: DataSet[LabeledVector] => Unit`
    +
    +### Predict
    +
    +KNN predicts for all subtypes of FlinkML's `Vector` the corresponding class label:
    +
    +* `predict[T <: Vector]: DataSet[T] => DataSet[(T, Array[Vector])]`, where the `(T, Array[Vector])` tuple
    +  corresponds to (testPoint, K-nearest training points)
    +
    +## Paremeters
    +The KNN implementation can be controlled by the following parameters:
    +
    +   <table class="table table-bordered">
    +    <thead>
    +      <tr>
    +        <th class="text-left" style="width: 20%">Parameters</th>
    +        <th class="text-center">Description</th>
    +      </tr>
    +    </thead>
    +
    +    <tbody>
    +      <tr>
    +        <td><strong>K</strong></td>
    +        <td>
    +          <p>
    +            Defines the number of nearest-neoghbors to search for.  That is, for each test point, the algorithm finds the K nearest neighbors in the training set
    +            (Default value: <strong>5</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong> DistanceMetric</strong></td>
    +        <td>
    +          <p>
    +            Sets the distance metric we use to calculate the distance between two points. If no metric is specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +            (Default value: <strong> EuclideanDistanceMetric </strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>Blocks</strong></td>
    +        <td>
    +          <p>
    +            Sets the number of blocks into which the input data will be split. This number should be set
    +            at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +            input [[DataSet]] is used as the number of blocks.
    +            (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>UseQuadTreeParam</strong></td>
    +        <td>
    +          <p>
    +             A boolean variable that whether or not to use a Quadtree to partition the training set to potentially simplify the KNN search.  If no value is specified, the code will automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well with the number of training and testing points, though poorly with the dimension.
    +            (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +      <tr>
    +        <td><strong>SizeHint</strong></td>
    +        <td>
    +          <p>Specifies whether the training set or test set is small to optimize the cross product operation needed for the KNN search.  If the training set is small this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL` if the test set is small.
    +             (Default value: <strong>None</strong>)
    +          </p>
    +        </td>
    +      </tr>
    +    </tbody>
    +  </table>
    +
    +## Examples
    +
    +{% highlight scala %}
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.classification.Classification
    +import org.apache.flink.ml.math.DenseVector
    +import org.apache.flink.ml.metrics.distances.
    +SquaredEuclideanDistanceMetric
    +
    +  val env = ExecutionEnvironment.getExecutionEnvironment
    +
    +  // prepare data
    +  val trainingSet = env.fromCollection(Classification.trainingData).map(_.vector)
    +  val testingSet = env.fromElements(DenseVector(0.0, 0.0))
    +
    + val knn = KNN()
    +    .setK(3)
    +    .setBlocks(10)
    +    .setDistanceMetric(SquaredEuclideanDistanceMetric())
    +    .setUseQuadTree(false)
    +    .setSizeHint(CrossHint.SECOND_IS_SMALL)
    +
    +  // run knn join
    +  knn.fit(trainingSet)
    +  val result = knn.predict(testingSet).collect()
    +
    +{% endhighlight %}
    +
    +For more details on the computing KNN with and without a d quadtree, here is a presentation:
    --- End diff --
    
    ... with and without _a quadtree_, ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-148115166
  
    @tillrohrmann scratch that, found a typo on my end that would result in a infinite loop....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146519265
  
    Thanks a lot @tillrohrmann, will make adjustments in the next few days.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093717
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +                      instance: KNN,
    +                      fitParameters: ParameterMap,
    +                      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +                                   instance: KNN,
    +                                   predictParameters: ParameterMap,
    +                                   input: DataSet[T]):
    +                                   DataSet[(FlinkVector, Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            // join input and training set
    +            val crossed = trainingSet.cross(inputSplit).mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    +                    Ordering.by(_._4))
    +
    +                  // use a quadtree if (4^dim)Ntest*log(Ntrain)
    +                  // < Ntest*Ntrain, and distance is Euclidean
    +                  val useQuadTree = resultParameters.get(UseQuadTreeParam).getOrElse(
    +                    training.values.head.size + math.log(math.log(training.values.length) /
    +                      math.log(4.0)) < math.log(training.values.length) / math.log(4.0) &&
    +                      (metric.isInstanceOf[EuclideanDistanceMetric] ||
    +                        metric.isInstanceOf[SquaredEuclideanDistanceMetric]))
    +
    +                  if (useQuadTree) {
    +                   knnQueryWithQuadTree(training.values, testing.values, k, metric,queue, out)
    +                  } else {
    +                    knnQueryBasic(training.values, testing.values, k, metric,queue, out)
    +                  }
    +                }
    +              }
    +            }
    +
    +            // group by input vector id and pick k nearest neighbor for each group
    +            val result = crossed.groupBy(2).sortGroup(3, Order.ASCENDING).reduceGroup {
    +              (iter, out: Collector[(FlinkVector, Array[FlinkVector])]) => {
    +                if (iter.hasNext) {
    +                  val head = iter.next()
    +                  val key = head._2
    +                  val neighbors: ArrayBuffer[FlinkVector] = ArrayBuffer(head._1)
    +
    +                  for ((vector, _, _, _) <- iter.take(k - 1)) {
    +                    // we already took a first element
    +                    neighbors += vector
    +                  }
    +
    +                  out.collect(key, neighbors.toArray)
    +                }
    +              }
    +            }
    +
    +            result
    +          case None => throw new RuntimeException("The KNN model has not been trained." +
    +            "Call first fit before calling the predict operation.")
    +
    +        }
    +      }
    +    }
    +  }
    +
    +  def knnQueryWithQuadTree[T <: FlinkVector](training: Vector[T],
    +                           testing: Vector[(Long, T)],
    +                           k: Int, metric: DistanceMetric,
    +                           queue: mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)],
    +                           out: Collector[(FlinkVector, FlinkVector, Long, Double)]) {
    +    /// find a bounding box
    +    val MinArr = Array.tabulate(training.head.size)(x => x)
    +    val MaxArr =Array.tabulate(training.head.size)(x => x)
    +
    +    val minVecTrain = MinArr.map(i => training.map(x => x(i)).min - 0.01)
    +    val minVecTest = MinArr.map(i => testing.map(x => x._2(i)).min - 0.01)
    +    val maxVecTrain = MaxArr.map(i => training.map(x => x(i)).min + 0.01)
    +    val maxVecTest = MaxArr.map(i => testing.map(x => x._2(i)).min + 0.01)
    +
    +    val MinVec = DenseVector(MinArr.map(i => Array(minVecTrain(i), minVecTest(i)).min))
    +    val MaxVec = DenseVector(MinArr.map(i => Array(maxVecTrain(i), maxVecTest(i)).max))
    +
    +    //default value of max elements/box is set to max(20,k)
    +    val maxPerBox = Array(k,20).max
    +    val trainingQuadTree = new QuadTree(MinVec, MaxVec, metric, maxPerBox)
    +
    +    for (v <- training) {
    +      trainingQuadTree.insert(v.asInstanceOf[FlinkVector])
    +    }
    +
    +    for  ((id, vector) <- testing) {
    +      //  Find siblings' objects and do local kNN there
    +      val siblingObjects =
    +        trainingQuadTree.searchNeighborsSiblingQueue(
    +          vector.asInstanceOf[FlinkVector])
    --- End diff --
    
    Same as above. We don't need casting.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146474169
  
    A more scalaesque solution for the `partitionBox` method without using vars and for loops would be
    
    ```
    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
        def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
          if (dim >= width.size) {
            box
          } else {
            val newBox = box.flatMap {
              vector =>
                val (up, down) = (vector.copy, vector)
                up.update(dim, up(dim) - width(dim) / 4)
                down.update(dim, down(dim) + width(dim) / 4)
    
                Seq(up, down)
            }
            partitionHelper(newBox, dim + 1)
          }
        }
    
        partitionHelper(Seq(center), 0)
      }
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Flink 1745

Posted by uce <gi...@git.apache.org>.
Github user uce commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-145449146
  
    Thanks for this impressive PR. A minor comment: could you edit the title of the PR to include more details than the issue ID (we can't do it because the ASF is managing the github account). It's hard to track the PRs otherwise. I would just go with the JIRA title: `[FLINK-1745] Add exact k-nearest-neighbours algorithm to machine learning library`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-153081375
  
    I see @chiwanpark , L191-270 contains mostly quad-tree related stuff, so that does make sense


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714066
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    --- End diff --
    
    Whitespace between parameter name and type `n: Node`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166286
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-171852976
  
    @danielblazevski, I think we can use `crossWithTiny` and `crossWithHuge` method to reduce shuffle cost. Best approach is that counting elements in both datasets and decide method to cross, but currently we simply add a parameter to decide this like following:
    
    ```scala
    import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    
    class KNN {
      // ...
    
      def setSizeHint(sizeHint: CrossHint): KNN = {
        parameters.add(SizeHint, sizeHint)
        this
      }
    
      // ...
    }
    
    object KNN {
      // ...
    
      case object SizeHint extends Parameter[CrossHint] {
        val defaultValue: Option[CrossHint] = None
      }
    
      // ...
    }
    ```
    
    And we can use the parameter in `predictValues` method:
    
    ```scala
    val crossTuned = sizeHint match {
      case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
        trainingSet.crossWithHuge(inputSplit)
      case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
        trainingSet.crossWithTiny(inputSplit)
      case _ => trainingSet.cross(inputSplit)
    }
    
    val crossed = crossTuned.mapPartition {
      // ...
    }
    
    // ...
    ```
    
    We have to decide the name of added parameter (`SizeHint`) and add documentation of explanation that which dataset is first (training) and which dataset is second (testing).
    
    By the way, there is no documentation for k-NN. Could you add the documentation to `docs/ml` directory? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092390
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    --- End diff --
    
    Please remove empty line between scala doc and function definition.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46122040
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    +      for (c <- node.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.nodeElements " + node.nodeElements)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param queryPoint
    +   */
    +  def insert(queryPoint: Vector){
    +    insertRecur(queryPoint,root)
    +  }
    +
    +  private def insertRecur(queryPoint: Vector,node: Node) {
    +    if (node.children == null) {
    +      if (node.nodeElements.length < maxPerBox ) {
    +        node.nodeElements += queryPoint
    +      } else{
    +        node.makeChildren()
    +        for (o <- node.nodeElements){
    +          insertRecur(o, node.children(node.whichChild(o)))
    +        }
    +        node.nodeElements.clear()
    +        insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +      }
    +    } else{
    +      insertRecur(queryPoint, node.children(node.whichChild(queryPoint)))
    +    }
    +  }
    +
    +  /**
    +   * Used to zoom in on a region near a test point for a fast KNN query.
    +   * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +   * which one computes the max distance D_s to queryPoint.  D_s is then used during the
    +   * kNN query to find all points within a radius D_s of queryPoint using searchNeighbors.
    +   * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the leaf
    +   * nodes of the minimal bounding box of the queryPoint. The priority of a leaf node
    +   * is an appropriate notion of the distance between the test point and the node,
    +   * which is defined by minDist(queryPoint),
    +   *
    +   * @param queryPoint
    +   * @return
    +   */
    +  def searchNeighborsSiblingQueue(queryPoint: Vector): ListBuffer[Vector] = {
    +    var ret = new ListBuffer[Vector]
    +    // edge case when the main box has not been partitioned at all
    +    if (root.children == null) {
    +      root.nodeElements.clone()
    +    } else {
    +      val nodeQueue = new PriorityQueue[(Double, Node)]()(Ordering.by(x => x._1))
    +      searchRecurSiblingQueue(queryPoint, root, nodeQueue)
    +
    +      var count = 0
    +      while (count < maxPerBox) {
    +        val dq = nodeQueue.dequeue()
    +        if (dq._2.nodeElements.nonEmpty) {
    +          ret ++= dq._2.nodeElements
    +          count += dq._2.nodeElements.length
    +        }
    +      }
    +      ret
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param queryPoint
    +   * @param node
    +   * @param nodeQueue defined in searchSiblingQueue, this stores nodes based on their
    +   *                  distance to node as defined by minDist
    +   */
    +  private def searchRecurSiblingQueue(queryPoint: Vector, node: Node,
    +                                      nodeQueue: PriorityQueue[(Double, Node)]) {
    +    if (node.children != null) {
    +      for (child <- node.children; if child.contains(queryPoint)) {
    +        if (child.children == null) {
    +          for (c <- node.children) {
    +            MinNodes(queryPoint,c,nodeQueue)
    --- End diff --
    
    Whitespaces between parameters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086830
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086829
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-160427679
  
    Hi @danielblazevski, I reviewed your updated pull request. There are only few problems to merge. Maybe after addressing them, we can merge this to master.
    
    First, there are still some codes with style inconsistency. Could you reformat all changes with your IDE? If you are using IntelliJ IDEA, you can do reformatting by pressing Cmd+Alt+Shift+L or Ctrl+Alt+Shift+L.
    
    About test case for `QuadTree`, adding a case to test `QuadTree` with non-supported distance metric would be good. And I think splitting test case is necessary. Many test run in a test case currently.
    
    Some implementation doesn't seem scalaesque. I suggest more scalaesque implementation. Could you review my suggestion?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r41526734
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,305 @@
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.Vector
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:ListBuffer[Double], maxVec:ListBuffer[Double],distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(c:ListBuffer[Double],L:ListBuffer[Double], var children:ListBuffer[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterLength(): (ListBuffer[Double], ListBuffer[Double]) = {
    +      (c, L)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < c(i) + L(i) / 2 && obj(i) + radius > c(i) - L(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        return true
    +      } else {
    +        return false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < c(i) - L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) + L(i) / 2, 2)
    +        } else if (obj(i) > c(i) + L(i) / 2) {
    +          minDist += math.pow(obj(i) - c(i) - L(i) / 2, 2)
    +        }
    +      }
    +      return minDist
    +    }
    +
    +    def whichChild(obj:Vector):Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1){
    +        if (obj(i) > c(i)) {
    +          count += Math.pow(2,i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      var cBuff = new ListBuffer[ListBuffer[Double]]
    +      cBuff += c
    +      var Childrennodes = new ListBuffer[Node]
    +      val cPart = partitionBox(cBuff,L,L.length)
    +      for (i <- cPart.indices){
    +        Childrennodes = Childrennodes :+ new Node(cPart(i), L.map(x => x/2.0), null)
    +
    +      }
    --- End diff --
    
    Maybe we can add a method such as `multiply(value: Double)` in `Vector` trait. But currently, there is no easy method for this situation. I think that following is best method currently:
    
    ```scala
    val mappedWidth = width match {
      case SparseVector(size, indices, data) =>
        val newData = data.map(_ / 2.0)
        SparseVector(size, indices, data)
      case DenseVector(data) =>
        val newData = data.map(_ / 2.0)
        DenseVector(data)
    }


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-184853357
  
    P.S. about rebasing, need to be careful, something went wrong the first time around.  I actually just started working on a new laptop, and started the git repo "from scratch" as follows:
    ```
    clone the master and FLINK-1745 branches of my fork of Flink
    checkout FLINK-1745, commit and push to origin (origin = my fork)
    ```
    
    I  set upstream to `origin`, is that a mistake?  Namely, when I push locally to GitHub, I set `upstream` to `origin`, namely I ran:
    ```
    git push --set-upstream origin FLINK-1745
    ```
    `origin` is my fork.  Should I re-do this by adding a new `remote` called `apache` and run
    ```
    git push --set-upstream apache FLINK-1745
    ```
    and then run the git commands you mentioned to rebase?  Want to be careful, making a re-basing mistake can be a nightmare to fix :-)  
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47166252
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093209
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    --- End diff --
    
    Please reformat this line. Remove a space after parenthesis and add spaces around `*`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r50647792
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.DataSetUtils._
    +//import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint).get
    --- End diff --
    
    Nice, that fixed it ! 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63709449
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    --- End diff --
    
    In `minDist`, an exception will be thrown.  I put this here in case the quadtree is used outside of knn.  This is also the only place where I need the assumption about being Euclidean (more details on this in your question below)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r63710693
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,354 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric, DistanceMetric,
    +EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k`-nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *       val trainingDS: DataSet[Vector] = ...
    +  *       val testingDS: DataSet[Vector] = ...
    +  *
    +  *       val knn = KNN()
    +  *         .setK(10)
    +  *         .setBlocks(5)
    +  *         .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *       knn.fit(trainingDS)
    +  *
    +  *       val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.UseQuadTreeParam]]
    +  * A boolean variable that whether or not to use a Quadtree to partition the training set
    +  * to potentially simplify the KNN search.  If no value is specified, the code will
    +  * automatically decide whether or not to use a Quadtree.  Use of a Quadtree scales well
    +  * with the number of training and testing points, though poorly with the dimension.
    +  * (Default value:  ```None```)
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.SizeHint]]
    +  * Specifies whether the training set or test set is small to optimize the cross
    +  * product operation needed for the KNN search.  If the training set is small
    +  * this should be `CrossHint.FIRST_IS_SMALL` and set to `CrossHint.SECOND_IS_SMALL`
    +  * if the test set is small.
    +  * (Default value:  ```None```)
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +   */
    +  def setUseQuadTree(useQuadTree: Boolean): KNN = {
    +    if (useQuadTree){
    +      require(parameters(DistanceMetric).isInstanceOf[SquaredEuclideanDistanceMetric] ||
    +        parameters(DistanceMetric).isInstanceOf[EuclideanDistanceMetric])
    +    }
    +    parameters.add(UseQuadTreeParam, useQuadTree)
    +    this
    +  }
    +
    +  /**
    +   * Parameter a user can specify if one of the training or test sets are small
    +   * @param sizeHint
    +   * @return
    +   */
    +  def setSizeHint(sizeHint: CrossHint): KNN = {
    +    parameters.add(SizeHint, sizeHint)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +  case object SizeHint extends Parameter[CrossHint] {
    +    val defaultValue: Option[CrossHint] = None
    +  }
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +  implicit def fitKNN[T <: FlinkVector : TypeInformation] = new FitOperation[KNN, T] {
    +    override def fit(
    +      instance: KNN,
    +      fitParameters: ParameterMap,
    +      input: DataSet[T]): Unit = {
    +      val resultParameters = instance.parameters ++ fitParameters
    +
    +      require(resultParameters.get(K).isDefined, "K is needed for calculation")
    +
    +      val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +      val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +      val inputAsVector = input.asInstanceOf[DataSet[FlinkVector]]
    +
    +      instance.trainingSet = Some(FlinkMLTools.block(inputAsVector, blocks, Some(partitioner)))
    +    }
    +  }
    +
    +  /** [[PredictDataSetOperation]] which calculates k-nearest neighbors of the given testing data
    +    * set.
    +    * @tparam T Subtype of [[Vector]]
    +    * @return The given testing data set with k-nearest neighbors
    +    */
    +  implicit def predictValues[T <: FlinkVector : ClassTag : TypeInformation] = {
    +    new PredictDataSetOperation[KNN, T, (FlinkVector, Array[FlinkVector])] {
    +      override def predictDataSet(
    +        instance: KNN,
    +        predictParameters: ParameterMap,
    +        input: DataSet[T]): DataSet[(FlinkVector,
    +        Array[FlinkVector])] = {
    +        val resultParameters = instance.parameters ++ predictParameters
    +
    +        instance.trainingSet match {
    +          case Some(trainingSet) =>
    +            val k = resultParameters.get(K).get
    +            val blocks = resultParameters.get(Blocks).getOrElse(input.getParallelism)
    +            val metric = resultParameters.get(DistanceMetric).get
    +            val partitioner = FlinkMLTools.ModuloKeyPartitioner
    +
    +            // attach unique id for each data
    +            val inputWithId: DataSet[(Long, T)] = input.zipWithUniqueId
    +
    +            // split data into multiple blocks
    +            val inputSplit = FlinkMLTools.block(inputWithId, blocks, Some(partitioner))
    +
    +            val sizeHint = resultParameters.get(SizeHint)
    +            val crossTuned = sizeHint match {
    +              case Some(hint) if hint == CrossHint.FIRST_IS_SMALL =>
    +                trainingSet.crossWithHuge(inputSplit)
    +              case Some(hint) if hint == CrossHint.SECOND_IS_SMALL =>
    +                trainingSet.crossWithTiny(inputSplit)
    +              case _ => trainingSet.cross(inputSplit)
    +            }
    +
    +            // join input and training set
    +            val crossed = crossTuned.mapPartition {
    +              (iter, out: Collector[(FlinkVector, FlinkVector, Long, Double)]) => {
    +                for ((training, testing) <- iter) {
    +                  val queue = mutable.PriorityQueue[(FlinkVector, FlinkVector, Long, Double)]()(
    --- End diff --
    
    Good point, that is leftover from refactoring when the queue was used in that scope.  Will change


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46086826
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    --- End diff --
    
    Added ScalaDocs, computes the right index for the child when searching through the tree -- the children are indexed, so instead of checking which box by looking at the coordinate values I compute the index.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46092375
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/KNN.scala ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn
    +
    +import org.apache.flink.api.common.operators.Order
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.utils._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.common._
    +import org.apache.flink.ml.math.{Vector => FlinkVector, DenseVector}
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +DistanceMetric, EuclideanDistanceMetric}
    +import org.apache.flink.ml.pipeline.{FitOperation, PredictDataSetOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import org.apache.flink.ml.nn.util.QuadTree
    +import scala.collection.mutable.ListBuffer
    +
    +import scala.collection.immutable.Vector
    +import scala.collection.mutable
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +/** Implements a k-nearest neighbor join.
    +  *
    +  * Calculates the `k` nearest neighbor points in the training set for each point in the test set.
    +  *
    +  * @example
    +  * {{{
    +  *     val trainingDS: DataSet[Vector] = ...
    +  *     val testingDS: DataSet[Vector] = ...
    +  *
    +  *     val knn = KNN()
    +  *       .setK(10)
    +  *       .setBlocks(5)
    +  *       .setDistanceMetric(EuclideanDistanceMetric())
    +  *
    +  *     knn.fit(trainingDS)
    +  *
    +  *     val predictionDS: DataSet[(Vector, Array[Vector])] = knn.predict(testingDS)
    +  * }}}
    +  *
    +  * =Parameters=
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.K]]
    +  * Sets the K which is the number of selected points as neighbors. (Default value: '''5''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.Blocks]]
    +  * Sets the number of blocks into which the input data will be split. This number should be set
    +  * at least to the degree of parallelism. If no value is specified, then the parallelism of the
    +  * input [[DataSet]] is used as the number of blocks. (Default value: '''None''')
    +  *
    +  * - [[org.apache.flink.ml.nn.KNN.DistanceMetric]]
    +  * Sets the distance metric we use to calculate the distance between two points. If no metric is
    +  * specified, then [[org.apache.flink.ml.metrics.distances.EuclideanDistanceMetric]] is used.
    +  * (Default value: '''EuclideanDistanceMetric()''')
    +  *
    +  */
    +
    +class KNN extends Predictor[KNN] {
    +
    +  import KNN._
    +
    +  var trainingSet: Option[DataSet[Block[FlinkVector]]] = None
    +
    +  /** Sets K
    +    * @param k the number of selected points as neighbors
    +    */
    +  def setK(k: Int): KNN = {
    +    require(k > 0, "K must be positive.")
    +    parameters.add(K, k)
    +    this
    +  }
    +
    +  /** Sets the distance metric
    +    * @param metric the distance metric to calculate distance between two points
    +    */
    +  def setDistanceMetric(metric: DistanceMetric): KNN = {
    +    parameters.add(DistanceMetric, metric)
    +    this
    +  }
    +
    +  /** Sets the number of data blocks/partitions
    +    * @param n the number of data blocks
    +    */
    +  def setBlocks(n: Int): KNN = {
    +    require(n > 0, "Number of blocks must be positive.")
    +    parameters.add(Blocks, n)
    +    this
    +  }
    +
    +  /**
    +   * Sets the Boolean variable that decides whether to use the QuadTree or not
    +    */
    +  def setUseQuadTree(UseQuadTree: Boolean): KNN = {
    +    parameters.add(UseQuadTreeParam, UseQuadTree)
    +    this
    +  }
    +
    +}
    +
    +object KNN {
    +
    +  case object K extends Parameter[Int] {
    +    val defaultValue: Option[Int] = Some(5)
    +  }
    +
    +  case object DistanceMetric extends Parameter[DistanceMetric] {
    +    val defaultValue: Option[DistanceMetric] = Some(EuclideanDistanceMetric())
    +  }
    +
    +  case object Blocks extends Parameter[Int] {
    +    val defaultValue: Option[Int] = None
    +  }
    +
    +  case object UseQuadTreeParam extends Parameter[Boolean] {
    +    val defaultValue: Option[Boolean] = None
    +  }
    +
    +
    +  def apply(): KNN = {
    +    new KNN()
    +  }
    +
    +  /** [[FitOperation]] which trains a KNN based on the given training data set.
    +    * @tparam T Subtype of [[org.apache.flink.ml.math.Vector]]
    +    */
    +
    --- End diff --
    
    Please remove empty line between scaladoc and function definition.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-159227040
  
    Hi @danielblazevski. Sorry for my late review. Things somehow piled up.
    
    I had some comments mainly concerning code style and formatting. It would be great if we could make it a bit more consistent with the rest of the code base. Other than that, it looks good. Thanks for your contribution.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r45714994
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.DistanceMetric
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec
    + * @param maxVec
    + */
    +class QuadTree(minVec:Vector, maxVec:Vector,distMetric:DistanceMetric){
    +  var maxPerBox = 20
    +
    +  class Node(center:Vector,width:Vector, var children:Seq[Node]) {
    +
    +    var objects = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(obj: Vector): Boolean = {
    +      overlap(obj, 0.0)
    +    }
    +
    +    /** Tests if obj is within a radius of the node
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(obj: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) - radius < center(i) + width(i) / 2 &&
    +          obj(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == obj.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if obj is near a node:  minDist is defined so that every point in the box
    +      * has distance to obj greater than minDist
    +      * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +      *
    +      * @param obj
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(obj: Vector, radius: Double): Boolean = {
    +      if (minDist(obj) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    def minDist(obj: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) + width(i) / 2, 2)
    +        } else if (obj(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(obj(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +      minDist
    +    }
    +
    +    def whichChild(obj: Vector): Int = {
    +
    +      var count = 0
    +      for (i <- 0 to obj.size - 1) {
    +        if (obj(i) > center(i)) {
    +          count += Math.pow(2, obj.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     *  Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center
    +     * @param width
    +     * @return
    +     *
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     *   simple printing of tree for testing/debugging
    +     */
    +  def printTree(){
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(n:Node){
    +    if(n.children != null) {
    +      for (c <- n.children){
    +        printTreeRecur(c)
    +      }
    +    }else{
    +      println("printing tree: n.objects " + n.objects)
    +    }
    +  }
    +
    +  /**
    +   * Recursively adds an object to the tree
    +   * @param obj
    +   */
    +  def insert(obj:Vector){
    +    insertRecur(obj,root)
    +  }
    +
    +  private def insertRecur(obj:Vector,n:Node) {
    +    if(n.children==null) {
    +      if(n.objects.length < maxPerBox )
    +      {
    +        n.objects += obj
    +      }
    +
    +      else{
    +        n.makeChildren()  ///make children nodes; place objects into them and clear node.objects
    +        for (o <- n.objects){
    +          insertRecur(o, n.children(n.whichChild(o)))
    +        }
    +        n.objects.clear()
    +        insertRecur(obj, n.children(n.whichChild(obj)))
    +      }
    +    } else{
    +      insertRecur(obj, n.children(n.whichChild(obj)))
    +    }
    +  }
    +
    +  /** Following methods are used to zoom in on a region near a test point for a fast KNN query.
    +    *
    +    * This capability is used in the KNN query to find k "near" neighbors n_1,...,n_k, from
    +    * which one computes the max distance D_s to obj.  D_s is then used during the
    +    * kNN query to find all points within a radius D_s of obj using searchNeighbors.
    +    * To find the "near" neighbors, a min-heap is defined on the leaf nodes of the quadtree.
    +    * The priority of a leaf node is an appropriate notion of the distance between the test
    +    * point and the node, which is defined by minDist(obj),
    +   *
    +   */
    +  private def subOne(tuple: (Double,Node)) = tuple._1
    +
    +  def searchNeighborsSiblingQueue(obj:Vector):ListBuffer[Vector] = {
    --- End diff --
    
    Scaladocs missing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r46093220
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    +  class Node(center: Vector, width: Vector, var children: Seq[Node]) {
    +
    +    val nodeElements = new ListBuffer[Vector]
    +
    +    /** for testing purposes only; used in QuadTreeSuite.scala
    +      *
    +      * @return center and width of the box
    +      */
    +    def getCenterWidth(): (Vector, Vector) = {
    +      (center, width)
    +    }
    +
    +    def contains(queryPoint: Vector): Boolean = {
    +      overlap(queryPoint, 0.0)
    +    }
    +
    +    /** Tests if queryPoint is within a radius of the node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def overlap(queryPoint: Vector, radius: Double): Boolean = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) - radius < center(i) + width(i) / 2 &&
    +          queryPoint(i) + radius > center(i) - width(i) / 2) {
    +          count += 1
    +        }
    +      }
    +
    +      if (count == queryPoint.size) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /** Tests if queryPoint is near a node
    +      *
    +      * @param queryPoint
    +      * @param radius
    +      * @return
    +      */
    +    def isNear(queryPoint: Vector, radius: Double): Boolean = {
    +      if (minDist(queryPoint) < radius) {
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    /**
    +     * used in error handling when computing minDist to make sure
    +     * distMetric is Euclidean or SquaredEuclidean
    +     * @param message
    +     */
    +    case class metricException(message: String) extends Exception(message)
    +
    +    /**
    +     * minDist is defined so that every point in the box
    +     * has distance to queryPoint greater than minDist
    +     * (minDist adopted from "Nearest Neighbors Queries" by N. Roussopoulos et al.)
    +     *
    +     * @param queryPoint
    +     * @return
    +     */
    +
    +    def minDist(queryPoint: Vector): Double = {
    +      var minDist = 0.0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) < center(i) - width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) + width(i) / 2, 2)
    +        } else if (queryPoint(i) > center(i) + width(i) / 2) {
    +          minDist += math.pow(queryPoint(i) - center(i) - width(i) / 2, 2)
    +        }
    +      }
    +
    +      if (distMetric.isInstanceOf[SquaredEuclideanDistanceMetric]) {
    +        minDist
    +      } else if (distMetric.isInstanceOf[EuclideanDistanceMetric]) {
    +        math.sqrt(minDist)
    +      } else{
    +        throw metricException(s" Error: metric must be Euclidean or SquaredEuclidean!")
    +      }
    +    }
    +
    +    /**
    +     * Finds which child queryPoint lies in.  node.children is a Seq[Node], and
    +     * whichChild finds the appropriate index of that Seq.
    +     * @param queryPoint
    +     * @return
    +     */
    +    def whichChild(queryPoint: Vector): Int = {
    +      var count = 0
    +      for (i <- 0 to queryPoint.size - 1) {
    +        if (queryPoint(i) > center(i)) {
    +          count += Math.pow(2, queryPoint.size -1 - i).toInt
    +        }
    +      }
    +      count
    +    }
    +
    +    def makeChildren() {
    +      val centerClone = center.copy
    +      val cPart = partitionBox(centerClone, width)
    +      val mappedWidth = 0.5*width.asBreeze
    +      children = cPart.map(p => new Node(p, mappedWidth.fromBreeze, null))
    +
    +    }
    +
    +    /**
    +     * Recursive function that partitions a n-dim box by taking the (n-1) dimensional
    +     * plane through the center of the box keeping the n-th coordinate fixed,
    +     * then shifting it in the n-th direction up and down
    +     * and recursively applying partitionBox to the two shifted (n-1) dimensional planes.
    +     *
    +     * @param center the center of the box
    +     * @param width a vector of lengths of each dimension of the box
    +     * @return
    +     */
    +    def partitionBox(center: Vector, width: Vector): Seq[Vector] = {
    +
    +      def partitionHelper(box: Seq[Vector], dim: Int): Seq[Vector] = {
    +        if (dim >= width.size) {
    +          box
    +        } else {
    +          val newBox = box.flatMap {
    +            vector =>
    +              val (up, down) = (vector.copy, vector)
    +              up.update(dim, up(dim) - width(dim) / 4)
    +              down.update(dim, down(dim) + width(dim) / 4)
    +
    +              Seq(up,down)
    +          }
    +          partitionHelper(newBox, dim + 1)
    +        }
    +      }
    +      partitionHelper(Seq(center), 0)
    +    }
    +  }
    +
    +
    +  val root = new Node( ((minVec.asBreeze + maxVec.asBreeze)*0.5).fromBreeze,
    +    (maxVec.asBreeze - minVec.asBreeze).fromBreeze, null)
    +
    +    /**
    +     * Simple printing of tree for testing/debugging
    +     */
    +  def printTree(): Unit = {
    +    printTreeRecur(root)
    +  }
    +
    +  def printTreeRecur(node: Node){
    +    if(node.children != null) {
    --- End diff --
    
    Need a space after `if`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1220#discussion_r47168358
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/nn/QuadTree.scala ---
    @@ -0,0 +1,340 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.ml.nn.util
    +
    +import org.apache.flink.ml.math.{Breeze, Vector}
    +import Breeze._
    +
    +import org.apache.flink.ml.metrics.distances.{SquaredEuclideanDistanceMetric,
    +EuclideanDistanceMetric, DistanceMetric}
    +
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.PriorityQueue
    +
    +/**
    + * n-dimensional QuadTree data structure; partitions
    + * spatial data for faster queries (e.g. KNN query)
    + * The skeleton of the data structure was initially
    + * based off of the 2D Quadtree found here:
    + * http://www.cs.trinity.edu/~mlewis/CSCI1321-F11/Code/src/util/Quadtree.scala
    + *
    + * Many additional methods were added to the class both for
    + * efficient KNN queries and generalizing to n-dim.
    + *
    + * @param minVec vector of the corner of the bounding box with smallest coordinates
    + * @param maxVec vector of the corner of the bounding box with smallest coordinates
    + * @param distMetric metric, must be Euclidean or squareEuclidean
    + * @param maxPerBox threshold for number of points in each box before slitting a box
    + */
    +class QuadTree(minVec: Vector, maxVec: Vector, distMetric: DistanceMetric, maxPerBox: Int){
    +
    --- End diff --
    
    In case later down the road, someone may want to define a quadtree for some other purpose and not need minDist, the kNN query only needs to enforce Euclidean/SquaredEuclidean because of the minDist function that defines the distance between a point and a box


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1745] Add exact k-nearest-neighbours al...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1220#issuecomment-146475752
  
    +1 for scalaesque solution. And we can add `@tailrec` annotation to `partitionHelper` function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---