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

[GitHub] flink pull request: [Flink-2030][ml]Online Histogram: Discrete and...

GitHub user sachingoel0101 opened a pull request:

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

    [Flink-2030][ml]Online Histogram: Discrete and Categorical

    This implements the Online Histograms for both categorical and continuous data. For continuous data, we emulate a continuous probability distribution which supports finding cumulative sum upto a particular value, and finding value upto a specific cumulative probability [Quantiles]. 
    For categorical fields, we emulate a probability mass function which supports finding the probability associated with every class.
    The continuous histogram follows this paper: http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    
    Note: This is a sub-task of https://issues.apache.org/jira/browse/FLINK-1727 which already has a PR pending review at https://github.com/apache/flink/pull/710.

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

    $ git pull https://github.com/sachingoel0101/flink online_histogram

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

    https://github.com/apache/flink/pull/861.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 #861
    
----
commit ec50b4bb4faf91570724b4aa79783936d0a9487f
Author: Sachin Goel <sa...@gmail.com>
Date:   2015-06-23T08:40:57Z

    Online Histogram: Discrete and Categorical, Test Suites included

----


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-132996034
  
    I'm currently reviewing it as well @chiwanpark. Please give me some more minutes :-)


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37528494
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    --- End diff --
    
    Bins represent the maximum number of numbers we are allowed to store. Since we are approximating a continuous distribution, we cannot store all the numbers. So, every number, when it arrives, updates the bin values to approximate itself better.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-141898541
  
    Sorry Sachin, my backlog is currently really long. But I also would like to
    take another look because you completely rewrote the whole implementation.
    Please bear a little bit longer with us. I will really try to review your
    PR so that your work was not in vain.
    
    On Fri, Sep 18, 2015 at 4:39 PM, Sachin Goel <no...@github.com>
    wrote:
    
    > Rebased to reflect the changes in the scala utility functions.
    > Travis failure on an unrelated error. Reported at jira id 2700.
    >
    > This has already undergone three reviews, and been open for almost four
    > months (as part of #710 <https://github.com/apache/flink/pull/710> too).
    > If this functionality is not required, I can close the PR; that'd however
    > automatically mean closing #710 <https://github.com/apache/flink/pull/710>
    > too since this is essential to implementing a Decision Tree. I'd prefer not
    > to though as I've spent a lot of time optimizing this.
    >
    > @chiwanpark <https://github.com/chiwanpark> @tillrohrmann
    > <https://github.com/tillrohrmann>
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/861#issuecomment-141471679>.
    >



---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132209035
  
    I have kept the documentation for statistics in this however. We can merge #1032 first and then this to maintain the illusion of cause-effect. :')


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137768373
  
    Re 1: Good change, implicit classes is the way to go here, no need for implicit conversion in the object.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132430686
  
    The only problem to merge this is invalid example in documentation. We need calling `collect()` method and `apply(0)` to execute statistics methods such as `quantile`, `sum`, ..., etc..
    
    Other things seem okay. :)


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132128284
  
    +1 for having two different methods by return type but we need more comments from @tillrohrmann, @thvasilo or other people because I'm not sure this is best approach.
    
    Would be okay the method names are `createContinuousHistogram` and `createCategoricalHistogram` if we decide create two methods?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533528
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    --- End diff --
    
    wouldn't it be better to have a dedicated count variable which is incremented when calling `add`? Then we would have to go over the complete list of items and add them up, when we want to know the total number of elements.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144192
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match{
    --- End diff --
    
    A space is needed between `match` and `{`.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533713
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    --- End diff --
    
    Maybe *Returns current number of elements in the bin with the given binIndex* is more descriptive


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33146188
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/CategoricalHistogram.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.math
    +
    +import java.util
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  */
    +class CategoricalHistogram(
    +    values: Array[Double],
    +    counts: Array[Int] = Array.ofDim[Int](0))
    +  extends OnlineHistogram
    +  with Serializable {
    +
    +  // sanity checks
    +  require(values.length > 0, "Number of classes should be a positive integer")
    +  require(counts.length == 0 || counts.length == values.length, "Counts should have the same " +
    +    "number" + " " + "of " + "entries as values")
    +  val data = new util.ArrayList[(Double, Int)](values.length)
    +  require(checkSanity, "All values in counts should be non-negative")
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    // search for the index where the value is equal to1 p
    +    val search = find(p)
    +    require(search >= 0, p + "is not present in the histogram")
    +    val currentVal = data.get(search)
    +    data.set(search, (currentVal._1, currentVal._2 + 1))
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): CategoricalHistogram = {
    +    h match{
    +      case h1: CategoricalHistogram => {
    +        require(B == bins && B == h.bins, "Size of the histograms and B should be equal")
    --- End diff --
    
    The semantics of the merge function change between the continuous and the categorical case.
    
    In the continuous case we are able to merge histograms with different number of buckets, but that is not possible
    in the discrete case. 
    
    Why not make it possible to merge discrete histograms as well? For h3 = h1.merge(h2)
    The buckets of the result h3 would be equal to the union of the buckets in h1 and h2. Am I missing something 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527730
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    --- End diff --
    
    I did mean *construed*, not *constructed* :')


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-131474758
  
    Hi @chiwanpark, thanks for reviewing this. This was my first time working in Scala, so I hope you'll forgive the slight mistakes (oversights, perhaps?). 
    I've tried to address most of your comments and left notes where I was not sure.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37532870
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    --- End diff --
    
    What are the *cumulative number of points at b*?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37531538
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/OnlineHistogram.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.statistics
    +
    +/** Base trait for an Online Histogram
    --- End diff --
    
    An Online Histogram is meant to approximate a data set with a distribution. 
    So, for example, for discrete valued data, we store counters of every class.
    For continuous data, we learn a distribution on a data set as more and more elements come along.
    
    It is online in the sense that we don't require the whole data set to build it. It is built incrementally, and for two parts of the data set, it can be merged to provide statistics for the combined set.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37557013
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match {
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    --- End diff --
    
    This should maybe be documented somewhere.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533038
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    --- End diff --
    
    Insert line break before.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534398
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    --- End diff --
    
    A lot of return statements = not very scalaesque


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137776491
  
    @thvasilo , I'm not sure I got that right. I hadn't tested the changed `DataSetUtils`. 
    I'm not sure how to handle the implicit types and class tags. Since you also think it's a good idea, would you like to take a shot at 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132552269
  
    Sounds good, just make sure to update the title and description of both PRs to reflect the current state 


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529025
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    --- End diff --
    
    Yes, I was thinking of doing that. But then, I thought this wouldn't make sense for a one-time usage. Should I?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37530410
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    --- End diff --
    
    Umm, let me see what can be done about this. This will perhaps mean getting rid of any lower and upper bounds altogether. 


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137777090
  
    If it is actually considered to be API breaking then we should open a new PR and JIRA to make sure the change is well documented. Let's see what Till thinks and I can create the JIRA and PR if needed.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-131523559
  
    Fixed.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137775231
  
    Also @tillrohrmann : Is changing the DataSetUtils structure considered an API breaking change and should we handle it differently?
    
    Any code using `import org.apache.flink.api.scala.DataSetUtils.utilsToDataSet` must now be changed to use `import org.apache.flink.api.scala.DataSetUtils._`


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-160428072
  
    Ping!


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-133707118
  
    @tillrohrmann, thanks for the brilliant suggestions. Using a `TreeMap` and `PriorityQueue` with invalidation, I've managed to bring down the complexity of the `add` and `merge` operations to logarithmic time. Further, `quantile` and `count` are also linear only, as they should be.
    Further, I've decided to put both the Histograms in the `accumulator` package since they're supposed to work like one anyway. There already was a *discrete* histogram in the `accumulator` package. The *continuous* one now resides in the same place.
    Also, the `DataSetUtils` class now contains functions to create histograms, providing access to these classes from the Java api itself instead of the ML library. That was needed to be done sooner or later. Flink-2274 actually asks for that. 
    @thvasilo @chiwanpark  


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527377
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    --- End diff --
    
    The idea is, histograms are meant to represent a probability distribution, and there is a very clear correspondence. A probability mass function corresponds to a discrete valued histogram and a probability density function corresponds to a continuous valued histogram.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132170286
  
    @thvasilo , @chiwanpark , I've made the required changes. 


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-122705520
  
    This now also incorporates [Flink-2379].


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137819
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/FieldStats.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.statistics
    +
    +import org.apache.flink.ml.statistics.FieldType._
    +
    +import scala.collection.mutable
    +
    +/** Class to represent Field statistics.
    +  *
    +  * =Parameters=
    +  * -[[fieldType]]:
    +  *   Type of this field, [[DISCRETE]] or [[CONTINUOUS]]
    +  *
    +  * For [[DISCRETE]] fields, [[entropy]], [[gini]] and [[categoryCounts]] are provided.
    +  * For [[CONTINUOUS]] fields, [[min]], [[max]], [[mean]] and [[variance]] are provided.
    +  *
    +  */
    +class FieldStats(val fieldType: FieldType) extends Serializable {
    +  // field parameters
    +  private [statistics] var _min: Double = _
    +  private [statistics] var _max: Double = _
    +  private [statistics] var _mean: Double = _
    +  private [statistics] var _variance: Double = _
    +  private [statistics] var _counts: mutable.HashMap[Double,Int] = _
    +
    +  //-------------------- Access methods ----------------------------//
    +  def min: Double = {
    +    exception(DISCRETE)
    +    _min
    +  }
    +
    +  def max: Double = {
    +    exception(DISCRETE)
    +    _max
    +  }
    +
    +  def mean: Double = {
    +    exception(DISCRETE)
    +    _mean
    +  }
    +
    +  def variance: Double = {
    +    exception(DISCRETE)
    +    _variance
    +  }
    +
    +  def categoryCounts: mutable.HashMap[Double,Int] = {
    +    exception(CONTINUOUS)
    +    _counts
    +  }
    +
    +  /**
    +   * Returns the entropy value for this [[DISCRETE]] field.
    +   */
    +  def entropy: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    3.322 * _counts.iterator.map(x => - (x._2 / total) * Math.log10(x._2 / total)).sum
    +  }
    +
    +  /**
    +   * Returns the Gini impurity for this [[DISCRETE]] field.
    +   */
    +  def gini: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    1 - _counts.iterator.map(x => (x._2 * x._2)).sum / (total * total)
    +  }
    +
    +  //------------------ Setter methods ------------------//
    +  private [statistics] def setContinuousParameters(
    --- End diff --
    
    Also `private[statistics]` would be better in 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114873908
  
    Hello Sachin, I have some comments regarding the discrete histogram. It seems that keeping the bucket count constant is incompatible with a "true" online histogram, which grows as values are added.
    
    What is the use case for the discrete histogram in that case? Do you think it makes sense for them to inherit from OnlineHistogram?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537319
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    --- End diff --
    
    Moreover, you can simply do `finalMap ++= 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137811
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/FieldStats.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.statistics
    +
    +import org.apache.flink.ml.statistics.FieldType._
    +
    +import scala.collection.mutable
    +
    +/** Class to represent Field statistics.
    +  *
    +  * =Parameters=
    +  * -[[fieldType]]:
    +  *   Type of this field, [[DISCRETE]] or [[CONTINUOUS]]
    +  *
    +  * For [[DISCRETE]] fields, [[entropy]], [[gini]] and [[categoryCounts]] are provided.
    +  * For [[CONTINUOUS]] fields, [[min]], [[max]], [[mean]] and [[variance]] are provided.
    +  *
    +  */
    +class FieldStats(val fieldType: FieldType) extends Serializable {
    +  // field parameters
    +  private [statistics] var _min: Double = _
    +  private [statistics] var _max: Double = _
    +  private [statistics] var _mean: Double = _
    +  private [statistics] var _variance: Double = _
    +  private [statistics] var _counts: mutable.HashMap[Double,Int] = _
    +
    +  //-------------------- Access methods ----------------------------//
    +  def min: Double = {
    +    exception(DISCRETE)
    +    _min
    +  }
    +
    +  def max: Double = {
    +    exception(DISCRETE)
    +    _max
    +  }
    +
    +  def mean: Double = {
    +    exception(DISCRETE)
    +    _mean
    +  }
    +
    +  def variance: Double = {
    +    exception(DISCRETE)
    +    _variance
    +  }
    +
    +  def categoryCounts: mutable.HashMap[Double,Int] = {
    +    exception(CONTINUOUS)
    +    _counts
    +  }
    +
    +  /**
    +   * Returns the entropy value for this [[DISCRETE]] field.
    +   */
    +  def entropy: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    3.322 * _counts.iterator.map(x => - (x._2 / total) * Math.log10(x._2 / total)).sum
    +  }
    +
    +  /**
    +   * Returns the Gini impurity for this [[DISCRETE]] field.
    +   */
    +  def gini: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    1 - _counts.iterator.map(x => (x._2 * x._2)).sum / (total * total)
    +  }
    +
    +  //------------------ Setter methods ------------------//
    +  private [statistics] def setContinuousParameters(
    +      min: Double,
    +      max: Double,
    +      mean: Double,
    +      variance: Double)
    +    : Unit = {
    +    exception(DISCRETE)
    +    _min = min
    +    _max = max
    +    _mean = mean
    +    _variance = variance
    +  }
    +
    +  private [statistics] def setDiscreteParameters(counts: mutable.HashMap[Double,Int]): Unit = {
    +    exception(CONTINUOUS)
    +    _counts = counts
    +  }
    +
    +  private def exception(checkFor: FieldType) = {
    +    if(fieldType == checkFor){
    +      throw new RuntimeException("Invalid access of data. Check field types.")
    +    }
    +  }
    --- End diff --
    
    Maybe If we check `FieldType` in each method (`min`, `max`, `mean`, ..., `gini`) with `Predef.assume` method, the user can understand what is the problem more properly. For example, when the user call `gini` method for continuous histogram, we can throw exception with detailed message such as "Gini impurity for continuous histogram is not supported.".


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33148440
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/CategoricalHistogram.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.math
    +
    +import java.util
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  */
    +class CategoricalHistogram(
    +    values: Array[Double],
    +    counts: Array[Int] = Array.ofDim[Int](0))
    +  extends OnlineHistogram
    +  with Serializable {
    +
    +  // sanity checks
    +  require(values.length > 0, "Number of classes should be a positive integer")
    +  require(counts.length == 0 || counts.length == values.length, "Counts should have the same " +
    +    "number" + " " + "of " + "entries as values")
    +  val data = new util.ArrayList[(Double, Int)](values.length)
    +  require(checkSanity, "All values in counts should be non-negative")
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    // search for the index where the value is equal to1 p
    +    val search = find(p)
    +    require(search >= 0, p + "is not present in the histogram")
    --- End diff --
    
    Why is this a requirement? The point of an online histogram is the ability to grow it incrementally.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33355362
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +120,23 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  private [ml] def createHistogram(data: DataSet[Double], bins: Int): DataSet[OnlineHistogram] = {
    --- End diff --
    
    Docstring required


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33356107
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,337 @@
    +/*
    + * 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.math
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  */
    +case class ContinuousHistogram(
    +    capacity: Int,
    +    min: Double,
    +    max: Double)
    +  extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else if (getValue(i) <= temp.getValue(j)) {
    --- End diff --
    
    Can't this be combined with the previous case? i.e.
    
    ```scala
    } else if (j >= n || getValue(i) <= temp.getValue(j)) {
                mergeList += data.apply(i)
                i = i + 1
    ```


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-118775803
  
    Since Till is very busy these days, I would ask @chiwanpark if he can take some time to review this, so that we can merge it if we get +1's.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-149491074
  
    @tillrohrmann , any updates regarding 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137937841
  
    Okay. Let me revert the commit and make this independently mergeable


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37540157
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    --- End diff --
    
    This is how merge is supposed to work. Both the histograms' bins are combined into a sorted list and then this is brought down to the required capacity. We cannot merge these elements one by one without having a sorted list first.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37557490
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    --- End diff --
    
    I think, some explaining comments in the code would be helpful for the understanding in the future.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#discussion_r38735611
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/accumulators/ContinuousHistogram.java ---
    @@ -0,0 +1,534 @@
    +/*
    + * 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.api.common.accumulators;
    +
    +import java.util.AbstractMap;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.PriorityQueue;
    +import java.util.Set;
    +import java.util.TreeMap;
    +
    +import static java.lang.Double.MAX_VALUE;
    +
    +/**
    + * A Histogram accumulator designed for Continuous valued data.
    + * It supports:
    + * -- {@link #quantile(double)}
    + * 		Computes a quantile of the data
    + * -- {@link #count(double)}
    + *		Computes number of items less than the given value in the data
    + * <p>
    + * A continuous histogram stores values in bins in sorted order and keeps their associated
    + * number of items. It is assumed that the items associated with every bin are scattered around
    + * it, half to the right and half to the left.
    + * <p>
    + * bin counters:  m_1    m_2    m_3    m_4    m_5    m_6
    + *                10     12     5      10     4      6
    + *                |  5   |  6   |  2.5 |  5   |  2   |
    + *             5  |  +   |  +   |   +  |  +   |  +   |  3
    + *                |  6   |  2.5 |   5  |  2   |  3   |
    + * - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    + * bin index:     1      2      3      4      5      6
    + * bin values:    v_1 <  v_2 <  v_3 <  v_4 <  v_5 <  v_6
    + * <p>
    + * The number of items between v_i and v_(i+1) is directly proportional to the area of
    + * trapezoid (v_i, v_(i+1), m_(i+1), m_i)
    + * <p>
    + * Adapted from Ben-Haim and Yom-Tov's
    + * <a href = "http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf">Streaming Decision Tree Algorithm's histogram</a>
    + */
    +public class ContinuousHistogram implements Accumulator<Double, TreeMap<Double, Integer>> {
    +
    +	protected TreeMap<Double, Integer> treeMap = new TreeMap<Double, Integer>();
    +
    +	protected long counter = 0;
    +
    +	private int bin;
    +
    +	private double lower;
    +
    +	private double upper;
    +
    +	private PriorityQueue<KeyDiff> diffQueue;
    +
    +	private HashMap<Double, KeyProps> keyUpdateTimes;
    +
    +	private long timestamp;
    +
    +	/**
    +	 * Creates a new Continuous histogram with the given number of bins
    +	 * Bins represents the number of values the histogram stores to approximate the continuous
    +	 * data set. The higher this value, the more we move towards an exact representation of data.
    +	 *
    +	 * @param numBins Number of bins in the histogram
    +	 */
    +	public ContinuousHistogram(int numBins) {
    +		if (numBins <= 0) {
    +			throw new IllegalArgumentException("Number of bins must be greater than zero");
    +		}
    +		bin = numBins;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue = new PriorityQueue<>();
    +		keyUpdateTimes = new HashMap<>();
    +		timestamp = 0;
    +	}
    +
    +	/**
    +	 * Consider using {@link #add(double)} for primitive double values to get better performance.
    +	 */
    +	@Override
    +	public void add(Double value) {
    +		add(value, 1);
    +	}
    +
    +	public void add(double value) {
    +		add(value, 1);
    +	}
    +
    +	@Override
    +	public TreeMap<Double, Integer> getLocalValue() {
    +		return this.treeMap;
    +	}
    +
    +	/**
    +	 * Get the total number of items added to this histogram.
    +	 * This is preserved across merge operations.
    +	 *
    +	 * @return Total number of items added to the histogram
    +	 */
    +	public long getTotal() {
    +		return counter;
    +	}
    +
    +	/**
    +	 * Get the current size of the {@link #treeMap}
    +	 *
    +	 * @return Size of the {@link #treeMap}
    +	 */
    +	public int getSize() {
    +		return treeMap.size();
    +	}
    +
    +	@Override
    +	public void resetLocal() {
    +		treeMap.clear();
    +		counter = 0;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue.clear();
    +		keyUpdateTimes.clear();
    +	}
    +
    +	@Override
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other) {
    +		fill(other.getLocalValue().entrySet());
    +	}
    +
    +	/**
    +	 * Merges the given other histogram into this histogram, with the number of bins in the
    +	 * merged histogram being {@code numBins}.
    +	 *
    +	 * @param other   Histogram to be merged
    +	 * @param numBins Bins in the merged histogram
    +	 */
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other, int numBins) {
    +		bin = numBins;
    +		merge(other);
    +	}
    +
    +	@Override
    +	public Accumulator<Double, TreeMap<Double, Integer>> clone() {
    +		ContinuousHistogram result = new ContinuousHistogram(bin);
    +		result.treeMap = new TreeMap<>(treeMap);
    +		result.counter = counter;
    +		result.lower = lower;
    +		result.upper = upper;
    +		// initialize all differences and key update times for the new histogram
    +		result.computeDiffs();
    +		return result;
    +	}
    +
    +	void add(double value, int count) {
    +		addValue(value, count);
    +		if (getSize() > bin) {
    +			mergeBins();
    +		}
    +	}
    +
    +	void fill(Set<Map.Entry<Double, Integer>> entries) {
    --- End diff --
    
    Same as 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114410824
  
    Thank you Sachin, I'll try to review this today.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37390659
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,98 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    **Discrete Histograms**: These histograms are formed on a data set `X:DataSet[Double]`
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A discrete histogram can be formed by calling `MLUtils.createDiscreteHistogram(X)`.
    +
    +### Data Statistics
    +
    + The `dataStats` function operates on a data set `X: DataSet[Vector]` and returns column-wise
    + statistics for `X`. Every field of `X` is allowed to be defined as either *discrete* or
    + *continuous*.
    + <br>
    + Statistics can be evaluated by calling `DataStats.dataStats(X)` or 
    + `DataStats.dataStats(X, discreteFields`). The latter is used when some fields are needed to be 
    --- End diff --
    
    Wrong position of backtick symbol. \`DataStats.dataStats(X, discreteFields)\`


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533768
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    --- End diff --
    
    Scala allows to write `data(bin)._2`


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534170
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    --- End diff --
    
    Shouldn't you check that before inserting the elements?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527033
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    --- End diff --
    
    What do you mean with *a* cumulative probability? Are there more than one for a given value set `X`?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529150
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    --- End diff --
    
    It would be good if you could add a rough outline of the functioning of the implemented algorithm 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37550949
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    --- End diff --
    
    How do you arrive at this formula?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37535755
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    +    val size: Int = bins
    +    if (size == 0 || p < getValue(0)) {
    +      return 0
    +    }
    +    for (i <- 0 to size - 2) {
    +      if (p >= getValue(i) && p < getValue(i + 1)) {
    +        return i + 1
    +      }
    +    }
    +    size
    +  }
    +
    +  /** Merges the closest two elements in the histogram
    +    * If we're over capacity, merge without thought
    +    * Otherwise, only merge when two elements have almost equal values. We're talking doubles, so
    +    * no exact equalities
    +    *
    +    * @return Whether we succeeded in merging any two elements
    +    */
    +  private def mergeElements(): Boolean = {
    --- End diff --
    
    Calling this method only merges two bins together. If you have to merge multiple bins because you're exceeding the capacity by more than 1, then you have to recompute the difference values over and over again even though they don't really change (only for two bin pairs). This is highly inefficient. Can't we write a method which merges elements until the merge condition is false? While merging, this method can reuse the calculated difference values.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-136789371
  
    @chiwanpark , can you review this 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-132991376
  
    Ah. Great. :) 
    I'll rebase #1032 immediately after that.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137379709
  
    +1 for adding a class for `Double` values.
    
    From my curiosity, Could `DiscreteHistogram` be used for decision tree? In the given paper, they used histogram based continuous 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137380847
  
    Yes. For Discrete fields, quantiles do not make sense. In the paper, they only cover the continuous fields, since the Discrete fields are more or less trivial to handle. [Unless there are too many categories].
    However, if we separate out the two histogram types, there is no need to implement a base class. The only shared functionality is the basic infra and fields. But the effective use of both is different as you pointed out. Or should I do that? I really can't settle on 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115654768
  
    Thank you Sachin, I'll go over again this today.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115204540
  
     How should I import a class in flink.ml.math from say, flink-java? I tried adding flink-staging as a dependency to pom.xml of flink-java but to no avail. I'm not terribly familiar with maven.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37538542
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    --- End diff --
    
    This started out as a work on Decision Tree and I needed them both to subclass from one trait. The only property they share is that they're Online, Histograms and effectively approximates two different types of data as a probability distribution. 


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37551371
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    --- End diff --
    
    This is painfully slow because count has a complexity of `O(n)`, thus making the `quantile` operation's complexity effectively quadratic. Why don't you simply add up the `getCounter` values yourself and keep track of the current accumulated value?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37551130
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    --- End diff --
    
    Could you elaborate a little bit on the used formula. I don't understand *walk along the trapezoidal 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37553105
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    --- End diff --
    
    Using an `ArrayBuffer` for the data storage is highly inefficient. Insert and remove operations have `O(n)` complexity. Since you perform these operations very often, it will be painfully slow. A better solution would be to use a `TreeSet` for example. A `TreeSet's` insert, remove and floor/ceiling function have complexity of `O(log(n))`.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137358805
  
    Fixed.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533127
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    --- End diff --
    
    insert line break before


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#discussion_r38735464
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/accumulators/ContinuousHistogram.java ---
    @@ -0,0 +1,534 @@
    +/*
    + * 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.api.common.accumulators;
    +
    +import java.util.AbstractMap;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.PriorityQueue;
    +import java.util.Set;
    +import java.util.TreeMap;
    +
    +import static java.lang.Double.MAX_VALUE;
    +
    +/**
    + * A Histogram accumulator designed for Continuous valued data.
    + * It supports:
    + * -- {@link #quantile(double)}
    + * 		Computes a quantile of the data
    + * -- {@link #count(double)}
    + *		Computes number of items less than the given value in the data
    + * <p>
    + * A continuous histogram stores values in bins in sorted order and keeps their associated
    + * number of items. It is assumed that the items associated with every bin are scattered around
    + * it, half to the right and half to the left.
    + * <p>
    + * bin counters:  m_1    m_2    m_3    m_4    m_5    m_6
    + *                10     12     5      10     4      6
    + *                |  5   |  6   |  2.5 |  5   |  2   |
    + *             5  |  +   |  +   |   +  |  +   |  +   |  3
    + *                |  6   |  2.5 |   5  |  2   |  3   |
    + * - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    + * bin index:     1      2      3      4      5      6
    + * bin values:    v_1 <  v_2 <  v_3 <  v_4 <  v_5 <  v_6
    + * <p>
    + * The number of items between v_i and v_(i+1) is directly proportional to the area of
    + * trapezoid (v_i, v_(i+1), m_(i+1), m_i)
    + * <p>
    + * Adapted from Ben-Haim and Yom-Tov's
    + * <a href = "http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf">Streaming Decision Tree Algorithm's histogram</a>
    --- End diff --
    
    Please remove 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#discussion_r38736695
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/accumulators/ContinuousHistogram.java ---
    @@ -0,0 +1,534 @@
    +/*
    + * 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.api.common.accumulators;
    +
    +import java.util.AbstractMap;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.PriorityQueue;
    +import java.util.Set;
    +import java.util.TreeMap;
    +
    +import static java.lang.Double.MAX_VALUE;
    +
    +/**
    + * A Histogram accumulator designed for Continuous valued data.
    + * It supports:
    + * -- {@link #quantile(double)}
    + * 		Computes a quantile of the data
    + * -- {@link #count(double)}
    + *		Computes number of items less than the given value in the data
    + * <p>
    + * A continuous histogram stores values in bins in sorted order and keeps their associated
    + * number of items. It is assumed that the items associated with every bin are scattered around
    + * it, half to the right and half to the left.
    + * <p>
    + * bin counters:  m_1    m_2    m_3    m_4    m_5    m_6
    + *                10     12     5      10     4      6
    + *                |  5   |  6   |  2.5 |  5   |  2   |
    + *             5  |  +   |  +   |   +  |  +   |  +   |  3
    + *                |  6   |  2.5 |   5  |  2   |  3   |
    + * - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    + * bin index:     1      2      3      4      5      6
    + * bin values:    v_1 <  v_2 <  v_3 <  v_4 <  v_5 <  v_6
    + * <p>
    + * The number of items between v_i and v_(i+1) is directly proportional to the area of
    + * trapezoid (v_i, v_(i+1), m_(i+1), m_i)
    + * <p>
    + * Adapted from Ben-Haim and Yom-Tov's
    + * <a href = "http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf">Streaming Decision Tree Algorithm's histogram</a>
    + */
    +public class ContinuousHistogram implements Accumulator<Double, TreeMap<Double, Integer>> {
    +
    +	protected TreeMap<Double, Integer> treeMap = new TreeMap<Double, Integer>();
    +
    +	protected long counter = 0;
    +
    +	private int bin;
    +
    +	private double lower;
    +
    +	private double upper;
    +
    +	private PriorityQueue<KeyDiff> diffQueue;
    +
    +	private HashMap<Double, KeyProps> keyUpdateTimes;
    +
    +	private long timestamp;
    +
    +	/**
    +	 * Creates a new Continuous histogram with the given number of bins
    +	 * Bins represents the number of values the histogram stores to approximate the continuous
    +	 * data set. The higher this value, the more we move towards an exact representation of data.
    +	 *
    +	 * @param numBins Number of bins in the histogram
    +	 */
    +	public ContinuousHistogram(int numBins) {
    +		if (numBins <= 0) {
    +			throw new IllegalArgumentException("Number of bins must be greater than zero");
    +		}
    +		bin = numBins;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue = new PriorityQueue<>();
    +		keyUpdateTimes = new HashMap<>();
    +		timestamp = 0;
    +	}
    +
    +	/**
    +	 * Consider using {@link #add(double)} for primitive double values to get better performance.
    +	 */
    +	@Override
    +	public void add(Double value) {
    +		add(value, 1);
    +	}
    +
    +	public void add(double value) {
    +		add(value, 1);
    +	}
    +
    +	@Override
    +	public TreeMap<Double, Integer> getLocalValue() {
    +		return this.treeMap;
    +	}
    +
    +	/**
    +	 * Get the total number of items added to this histogram.
    +	 * This is preserved across merge operations.
    +	 *
    +	 * @return Total number of items added to the histogram
    +	 */
    +	public long getTotal() {
    +		return counter;
    +	}
    +
    +	/**
    +	 * Get the current size of the {@link #treeMap}
    +	 *
    +	 * @return Size of the {@link #treeMap}
    +	 */
    +	public int getSize() {
    +		return treeMap.size();
    +	}
    +
    +	@Override
    +	public void resetLocal() {
    +		treeMap.clear();
    +		counter = 0;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue.clear();
    +		keyUpdateTimes.clear();
    +	}
    +
    +	@Override
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other) {
    +		fill(other.getLocalValue().entrySet());
    +	}
    +
    +	/**
    +	 * Merges the given other histogram into this histogram, with the number of bins in the
    +	 * merged histogram being {@code numBins}.
    +	 *
    +	 * @param other   Histogram to be merged
    +	 * @param numBins Bins in the merged histogram
    +	 */
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other, int numBins) {
    +		bin = numBins;
    +		merge(other);
    +	}
    +
    +	@Override
    +	public Accumulator<Double, TreeMap<Double, Integer>> clone() {
    +		ContinuousHistogram result = new ContinuousHistogram(bin);
    +		result.treeMap = new TreeMap<>(treeMap);
    +		result.counter = counter;
    +		result.lower = lower;
    +		result.upper = upper;
    +		// initialize all differences and key update times for the new histogram
    +		result.computeDiffs();
    +		return result;
    +	}
    +
    +	void add(double value, int count) {
    --- End diff --
    
    Ah. Yes. This is not required. This was protected when both histograms were derived from the same base class. I'll change 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37535831
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    --- End diff --
    
    A little bit more details 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527594
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    **Discrete Histograms**: These histograms are formed on a data set `X:DataSet[Double]`
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A discrete histogram can be formed by calling `MLUtils.createDiscreteHistogram(X)`.
    --- End diff --
    
    Is there a reason why the online histogram and the discrete histogram are created differently?


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137605
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    if (data.get(c).isEmpty) {
    +      require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +      data.put(c, 1)
    +    } else {
    +      data.update(c, data.get(c).get + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): CategoricalHistogram = {
    +    h match {
    +      case h1: CategoricalHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          if (finalMap.get(x._1).isEmpty) {
    +            finalMap.put(x._1, x._2)
    +          } else {
    +            finalMap.update(x._1, x._2 + finalMap.get(x._1).get)
    +          }
    +        })
    --- End diff --
    
    As I said above, we can reduce calling `get(x._1)` by using pattern matching for `finalMap.get(x._1)`.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114896951
  
    > Actually, how about making the Categorical Histogram completely online too? We could just take a numClasses there too, with a semantic difference that in this case the number of buckets would be absolute, with a count and real value associated with each bucket. The internal data structure would be a hashmap now, with the maximum allowed size numClasses.
    
    I like this approach.
    
    > Adding a Utility method does certainly make sense. User will be supposed to provide an argument depicting whether the values in DataSet[Double] are continuous or discrete.
    
    I think it's enough to assume that the values are continuous and use that code.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37543763
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    --- End diff --
    
    Why do we add half of the `getCounter(index)` bin here 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132261492
  
    Sorry to be a nitpick, but if we are going to split the PR then the documentation should be split accordingly, we can merge the column-wise statistics once this one is merged. I know this is taking very long to merge, but it is better if we do things properly and not rush them.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37538887
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    --- End diff --
    
    I couldn't find `sum` in `ContinuousHistogram`. Did you mean `count`?


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144171
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,100 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. <strong>Continuous Histograms</strong>: These histograms are formed on a data set `X:
    +   DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, <i>scaled</i> probability].
    +   <br>
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    <strong>Categorical Histograms</strong>: These histograms are formed on a data set `X:DataSet[Double]` 
    --- End diff --
    
    `<strong>` tag


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33357307
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/OnlineHistogram.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.math
    +
    +/** Base trait for an Online Histogram
    +  *
    +  */
    +trait OnlineHistogram extends Serializable {
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  def bins: Int
    +
    +  /** Adds a new instance
    --- End diff --
    
    Change to "Adds a new item to the histogram"


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33355896
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,337 @@
    +/*
    + * 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.math
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  */
    +case class ContinuousHistogram(
    +    capacity: Int,
    +    min: Double,
    +    max: Double)
    +  extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    --- End diff --
    
    Can change to `mergeList += ((temp.getValue(j), temp.getCounter(j)))` to improve readability.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132187018
  
    Olrite. I'll be back in a few. :)


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132494738
  
    Good! But because there is another issue (FLINK-2379) covered the column-wise statistics, I think that spliting them would be better.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-132554412
  
    Updated the title and description for both. 
    I hope I can get these two in with today's commits since there are no more issues. I need both of these to finalize the Decision Tree PR #708. It has been open for too long.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534692
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    +    val size: Int = bins
    +    if (size == 0 || p < getValue(0)) {
    +      return 0
    +    }
    +    for (i <- 0 to size - 2) {
    +      if (p >= getValue(i) && p < getValue(i + 1)) {
    +        return i + 1
    +      }
    +    }
    +    size
    +  }
    +
    +  /** Merges the closest two elements in the histogram
    +    * If we're over capacity, merge without thought
    +    * Otherwise, only merge when two elements have almost equal values. We're talking doubles, so
    +    * no exact equalities
    +    *
    +    * @return Whether we succeeded in merging any two elements
    +    */
    +  private def mergeElements(): Boolean = {
    +    val size: Int = bins
    +    var minDiffIndex: Int = -1
    +    var minDiff: Double = MaxValue
    +    for (i <- 0 to size - 2) {
    --- End diff --
    
    insert line break before


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527987
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    +      var statistics: (Double, Double) = _
    +
    +      override def open(configuration: Configuration): Unit = {
    +        statistics = getRuntimeContext.getBroadcastVariable(HISTOGRAM_STATS).get(0)
    +        val minimum = statistics._1
    +        val maximum = statistics._2
    +        statistics = (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +      }
    +
    +      override def mapPartition(
    +          values: java.lang.Iterable[Double],
    +          out: Collector[ContinuousHistogram])
    +        : Unit = {
    +        val localHistogram = new ContinuousHistogram(bins, statistics._1, statistics._2)
    +        val iterator = values.iterator()
    +        while (iterator.hasNext) {
    +          localHistogram.add(iterator.next())
    +        }
    +        out.collect(localHistogram)
    +      }
    +    })
    +      .withBroadcastSet(stats, HISTOGRAM_STATS)
    +      .reduce((x, y) => x.merge(y, bins))
    --- End diff --
    
    same 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527131
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    --- End diff --
    
    Typo: constructed


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37555537
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    --- End diff --
    
    According to the Scala source: `final val MinValue = -java.lang.Double.MAX_VALUE`


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537658
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match {
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        val finalSize = if (B > 0) B else finalMap.size
    +        val ret = new DiscreteHistogram(finalSize)
    +        ret.loadData(finalMap.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a discrete histogram is allowed to be merged with a " +
    +          "discrete histogram")
    +    }
    +  }
    +
    +  /** Number of elements in category c
    +    *
    +    * @return Number of points in category c
    +    */
    +  def count(c: Double): Int = {
    +    data.get(c) match {
    +      case None => 0
    +      case Some(value) => value
    +    }
    +  }
    +
    +  /** Returns the total number of elements in the histogram
    +    *
    +    * @return total number of elements added so far
    +    */
    +  override def total: Int = data.values.sum
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    --- End diff --
    
    Knowing that this is the discrete histogram would be 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-129336972
  
    Hi, I just discovered the review request. I'll review this PR soon. Because I'm busy in working for my graduation essay, maybe I can start reviewing on weekend.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37557797
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    --- End diff --
    
    Yes. I'm working on this. The best way to optimize this would be to change the signature and have `merge` actually modify the current histogram. This would be minimally expensive in terms of copying the data. But comes at the cost of having to duplicate the quantities `min`, `max` and `capacity`.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37554624
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    --- End diff --
    
    Ah yes. Thanks for pointing that out. This will make it quite fast.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37554683
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    --- End diff --
    
    A TreeSet won't suffice here since we need the values to be sorted.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37557297
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    --- End diff --
    
    For a data set of vectors, we need to construct histograms over all dimensions. If the fields are not all continuous or all discrete, we need a common parent class.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137376194
  
    Hi, I just reviewed the updated PR. Sorry for late.
    
    Your implementation is nice. It seems acting like I expected. But I have to check the following:
    
    I'm concerned about changing `Histogram` class. It causes API breaking. I'm not sure that this breaking is necessary. Reverting the changes of `Histogram` would be better. Because there are many differences between `ContinuousHistogram` and `DiscreteHistogram`, we don't need to create base class for them.
    
    I'm sorry about consuming time to merge this PR. It seems almost arrived the goal. Cheer 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137714631
  
    Hello, my 2c: This PR should include docs and a wrapper for the Scala API. We can also do this with a separate issue but it would be best if we merge as a more complete package.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137772917
  
    I think it would be hard to do otherwise. `java.lang.Double`'s are objects but `scala.Double`'s are not from what I understand. In this case duplicating the code might not be so bad.
    Perhaps @tillrohrmann has a better 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137821
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/FieldStats.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.statistics
    +
    +import org.apache.flink.ml.statistics.FieldType._
    +
    +import scala.collection.mutable
    +
    +/** Class to represent Field statistics.
    +  *
    +  * =Parameters=
    +  * -[[fieldType]]:
    +  *   Type of this field, [[DISCRETE]] or [[CONTINUOUS]]
    +  *
    +  * For [[DISCRETE]] fields, [[entropy]], [[gini]] and [[categoryCounts]] are provided.
    +  * For [[CONTINUOUS]] fields, [[min]], [[max]], [[mean]] and [[variance]] are provided.
    +  *
    +  */
    +class FieldStats(val fieldType: FieldType) extends Serializable {
    +  // field parameters
    +  private [statistics] var _min: Double = _
    +  private [statistics] var _max: Double = _
    +  private [statistics] var _mean: Double = _
    +  private [statistics] var _variance: Double = _
    +  private [statistics] var _counts: mutable.HashMap[Double,Int] = _
    +
    +  //-------------------- Access methods ----------------------------//
    +  def min: Double = {
    +    exception(DISCRETE)
    +    _min
    +  }
    +
    +  def max: Double = {
    +    exception(DISCRETE)
    +    _max
    +  }
    +
    +  def mean: Double = {
    +    exception(DISCRETE)
    +    _mean
    +  }
    +
    +  def variance: Double = {
    +    exception(DISCRETE)
    +    _variance
    +  }
    +
    +  def categoryCounts: mutable.HashMap[Double,Int] = {
    +    exception(CONTINUOUS)
    +    _counts
    +  }
    +
    +  /**
    +   * Returns the entropy value for this [[DISCRETE]] field.
    +   */
    +  def entropy: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    3.322 * _counts.iterator.map(x => - (x._2 / total) * Math.log10(x._2 / total)).sum
    +  }
    +
    +  /**
    +   * Returns the Gini impurity for this [[DISCRETE]] field.
    +   */
    +  def gini: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    1 - _counts.iterator.map(x => (x._2 * x._2)).sum / (total * total)
    +  }
    +
    +  //------------------ Setter methods ------------------//
    +  private [statistics] def setContinuousParameters(
    +      min: Double,
    +      max: Double,
    +      mean: Double,
    +      variance: Double)
    +    : Unit = {
    +    exception(DISCRETE)
    +    _min = min
    +    _max = max
    +    _mean = mean
    +    _variance = variance
    +  }
    +
    +  private [statistics] def setDiscreteParameters(counts: mutable.HashMap[Double,Int]): Unit = {
    --- End diff --
    
    Also `private[statistics]` would be better in 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556917
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    --- End diff --
    
    But they only share the concept of an online histogram. For what do you they have to inherit from the same trait practically?


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144068
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +120,31 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[DataSet]] of [[OnlineHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required. Zero for [[CategoricalHistogram]]
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[DataSet]] of [[OnlineHistogram]]
    +    */
    +  private [ml] def createHistogram(data: DataSet[Double], bins: Int): DataSet[OnlineHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max){
    +      (minimum,maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +    val ret: DataSet[OnlineHistogram] = data.mapWithBcVariable(stats){ (x, statistics) =>
    +      if(bins > 0){
    --- End diff --
    
    `if (bins > 0) {` would be better.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37555313
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    --- End diff --
    
    Then you're inconsistent with your usage of terms with respect to here and the documentation.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533263
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    --- End diff --
    
    return statement


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37536712
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    --- End diff --
    
    what is *h*? Would be easier to understand if you write *Merges this histogram with the given histogram h. The result is a new histogram* .


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37532792
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    --- End diff --
    
    Isn't a probability in [0,1]? But this method returns an int.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37557444
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    --- End diff --
    
    What do you mean with mapping it to the area of the trapezoid? In the end we don't want to obtain an area but a number of elements, right?


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137647
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    if (data.get(c).isEmpty) {
    +      require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +      data.put(c, 1)
    +    } else {
    +      data.update(c, data.get(c).get + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): CategoricalHistogram = {
    +    h match {
    +      case h1: CategoricalHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          if (finalMap.get(x._1).isEmpty) {
    +            finalMap.put(x._1, x._2)
    +          } else {
    +            finalMap.update(x._1, x._2 + finalMap.get(x._1).get)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        val countBuffer = new mutable.ArrayBuffer[(Double, Int)]()
    +        finalMap.iterator.foreach(x => countBuffer += ((x._1, x._2)))
    +        var finalSize = finalMap.size
    +        if (B > 0) {
    +          finalSize = B
    +        }
    +        val ret = new CategoricalHistogram(finalSize)
    +        ret.loadData(countBuffer.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a categorical histogram is allowed to be merged with a " +
    +          "categorical histogram")
    +    }
    +  }
    +
    +  /** Number of elements in category c
    +    *
    +    * @return Number of points in category c
    +    */
    +  def count(c: Double): Int = {
    +    if (data.get(c).isEmpty) {
    +      return 0
    +    }
    +    data.get(c).get
    +  }
    --- End diff --
    
    I think following is better than current implementation.
    
    ```scala
    def count(c: Double): Int = data.get(c) match {
      case None => 0
      case Some(v) => v
    }
    ```


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37555399
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/OnlineHistogram.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.statistics
    +
    +/** Base trait for an Online Histogram
    --- End diff --
    
    Then you should maybe document this in the code.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37533933
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    --- End diff --
    
    Maybe we could add that this is a histogram somewhere in the string.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37550922
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    --- End diff --
    
    How do you arrive at this formula?


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114893705
  
    Actually, how about making the Categorical Histogram completely online too? We could just take a numClasses there too, with a semantic difference that in this case the number of buckets would be absolute, with a count and _real_ value associated with each bucket. The internal data structure would be a hashmap now, with the maximum allowed size numClasses.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37138026
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  *   Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  *   Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  *   Upper limit on the elements
    +  */
    +case class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += (data.apply(i))
    --- End diff --
    
    Unnecessary parenthesis


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137752
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  *   Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  *   Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  *   Upper limit on the elements
    +  */
    +case class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += (data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    counts.iterator.foreach(x => data += ((x._1, x._2)))
    --- End diff --
    
    `data ++= counts` is same as 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37530125
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    --- End diff --
    
    line break


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37554750
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    --- End diff --
    
    Then this should be written there. Some of my remarks are more of a rhetorical character to point you to what I think is missing there, especially when it concerns the documentation.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-133870941
  
    +1 for moving histogram functions into `DataSetUtils`. It would be helpful for range partitioning. I'll review this in next 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37141076
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,108 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    + <ul>
    +  <li>
    +   <strong>Continuous Histograms</strong>: These histograms are formed on a data set `X: DataSet[Double]` 
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, <i>scaled</i> probability].
    +   <br>
    +    A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +  </li>
    +  <li>
    +    <strong>Categorical Histograms</strong>: These histograms are formed on a data set `X:DataSet[Double]` 
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A categorical histogram can be formed by calling `X.createHistogram(0)`.
    +  </li>
    + </ul>
    +
    +### Data Statistics
    +
    + The `dataStats` function operates on a data set `X: DataSet[Vector]` and returns column-wise
    + statistics for `X`. Every field of `X` is allowed to be defined as either <i>discrete</i> or
    + <i>continuous</i>.
    + <br>
    + Statistics can be evaluated by calling `DataStats.dataStats(X)` or 
    + `DataStats.dataStats(X, discreteFields`). The latter is used when some fields are needed to be 
    + declared discrete-valued, and is provided as an array of indices of fields which are discrete.
    + <br>
    + The following information is available as part of `DataStats`:
    + <ul>
    +    <li>Number of elements in `X`</li>
    +    <li>Dimension of `X`</li>
    +    <li>Column-wise statistics where for discrete fields, we report counts for each category, and
    +     the Gini impurity and Entropy of the field, while for continuous fields, we report the
    +     minimum, maximum, mean and variance.
    +    </li>
    + </ul>
    +
    +## Examples
    +
    +{% highlight scala %}
    +
    +import org.apache.flink.ml.statistics._
    +import org.apache.flink.ml._
    +
    +val X: DataSet[Double] = ...
    +// Create continuous histogram
    +val histogram = X.createHistogram(5)     // creates a histogram with five bins
    +histogram.quantile(0.3)                  // returns the 30th quantile
    +histogram.sum(4)                         // returns number of elements less than 4
    --- End diff --
    
    You're right. I'm wondering if we should provide two methods for this purpose. Otherwise user will need to cast into the appropriate class.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37532474
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    --- End diff --
    
    There is no `MinValue` in the sense it is needed here. A `MinValue` would be the smallest positive number which can be represented.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-132987549
  
    Looks good to merge. If there is no opposition in few hours, I'll merge this to master.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132137740
  
    I would suggest, we have both functions public in the MLUtils class. Further, we provide a pimp-my-class function for only Continuous histogram.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556566
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    +      var statistics: (Double, Double) = _
    +
    +      override def open(configuration: Configuration): Unit = {
    +        statistics = getRuntimeContext.getBroadcastVariable(HISTOGRAM_STATS).get(0)
    +        val minimum = statistics._1
    +        val maximum = statistics._2
    +        statistics = (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +      }
    +
    +      override def mapPartition(
    +          values: java.lang.Iterable[Double],
    +          out: Collector[ContinuousHistogram])
    +        : Unit = {
    +        val localHistogram = new ContinuousHistogram(bins, statistics._1, statistics._2)
    +        val iterator = values.iterator()
    +        while (iterator.hasNext) {
    +          localHistogram.add(iterator.next())
    +        }
    +        out.collect(localHistogram)
    +      }
    +    })
    +      .withBroadcastSet(stats, HISTOGRAM_STATS)
    +      .reduce((x, y) => x.merge(y, bins))
    +  }
    +
    +  /** Create a [[DiscreteHistogram]] from the input data
    +    *
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[DiscreteHistogram]] over the data
    +    */
    +  def createDiscreteHistogram(data: DataSet[Double]): DataSet[DiscreteHistogram] = {
    +    data.mapPartition(new RichMapPartitionFunction[Double, DiscreteHistogram] {
    --- End diff --
    
    Yeah. Already done this. Will push with the rest of the changes.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37558286
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    --- End diff --
    
    Btw: You're not calculating the area of the trapezoid because you divide by `(getValue(index + 1) - getValue(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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37558043
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    --- End diff --
    
    Have you read the JavaDocs of a `TreeSet`? In the first line it's written 
    
    > The elements are ordered using their natural ordering, or by a Comparator provided at set creation time, depending on which constructor is used


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132186504
  
    I'm also inclined to use Discrete.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132192126
  
    Hello, I was wondering when the column-wise statistics were added to this PR and whether it makes sense to add them here, or create a new issue and PR for them.
    
    Is the functionality between the two features tightly coupled?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37558122
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    --- End diff --
    
    I meant mapping area as in, taking ratio of area to the number of elements. The assumption is that elements are uniformly distributed between two different bins. This is why the line connecting m_x and m_y is assumed to be linear.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137764885
  
    ```scala
    
      def createDiscreteHistogram: DataSet[DiscreteHistogram] = {
        wrap(jutils.DataSetUtils.createDiscreteHistogram(
          self.map(x => new java.lang.Double(x)).javaSet))
      }
    
    ```
    
    Would something like this 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-138324568
  
    Will do.
    
    On Mon, Sep 7, 2015 at 4:58 PM, Sachin Goel <no...@github.com>
    wrote:
    
    > @tillrohrmann <https://github.com/tillrohrmann> , can you do a final
    > review of this?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/861#issuecomment-138318843>.
    >



---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132186102
  
    Ah yes. You're right. I will change the names. Which do you think is the better name? I'm in favor of Discrete.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114946436
  
    Yes, we should avoid mixing java and Scala of possible.
    
    -- 
    Sent from a mobile device. May contain autocorrect errors.
    On Jun 24, 2015 5:09 PM, "Sachin Goel" <no...@github.com> wrote:
    
    > Okay. Sure.
    > I will update the make the Discrete version online first.
    > Should I try to explicitly use Scala library data structures instead of
    > Java?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/861#issuecomment-114901644>.
    >



---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37530184
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    --- End diff --
    
    could we give the other histogram a more meaningful name other than `temp`?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37551827
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    --- End diff --
    
    Referring to the previous figure, this time we require that,
    
    m_x ( b - x) + (m_b - m_x) * (b - x) / 2 = (q * total - m_x)
    Further, m_b = (b - x) * (m_y - m_x) / ( y - x)


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114881148
  
    So in order to make this a feature that we can merge, we should add some functionality that goes beyond the decision tree use case.
    
    My thoughts were that we use the continuous histogram code in order to add a pimp-your-class style histogram(numBuckets: Int) function to a DataSet[Double].
    
    This will cover in part [FLINK-1127](https://issues.apache.org/jira/browse/FLINK-1127). The design should be similar to [this](https://github.com/apache/flink/pull/832/files#diff-04a913384132b325b844f41b9545b2cf) pending PR, where the histogram function would be added to DataSetUtils, for DataSets[Double]. How does that sound?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37532616
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    --- End diff --
    
    Yes. I couldn't figure out any other way. Usually, one would use a break statement here but scala doesn't have that.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534088
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    --- End diff --
    
    Improve 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#discussion_r38619828
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java ---
    @@ -248,6 +251,58 @@ public void mapPartition(Iterable<T> values, Collector<Tuple2<Long, T>> out) thr
     			input.getType(), sampleInCoordinator, callLocation);
     	}
     
    +	/**
    +	 * Creates a {@link org.apache.flink.api.common.accumulators.DiscreteHistogram} from the data set
    +	 *
    +	 * @param data Discrete valued data set
    +	 * @return A histogram over data
    +	 */
    +	public static DataSet<DiscreteHistogram> createDiscreteHistogram(DataSet<Double> data) {
    +		return data.mapPartition(new RichMapPartitionFunction<Double, DiscreteHistogram>() {
    +			@Override
    +			public void mapPartition(Iterable<Double> values, Collector<DiscreteHistogram> out)
    +					throws Exception {
    --- End diff --
    
    Unnecessary new 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37547739
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    --- End diff --
    
    So basically, m_x ( y - x )  + (y - x) * ( m_y - m_x) / 2  corresponds to (m_x + m_y) / 2 and then we map this to the area of trapezoid upto b.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137690
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  *   Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  *   Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  *   Upper limit on the elements
    +  */
    +case class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += (data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    --- End diff --
    
    Calculating total count of the histogram is already implemented in `total` method. We can remove between line 108 and line 111.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137382016
  
    Okay. Please take a look now. 
    [There is an unintentional formatting introduced by IDE in Histogram.java. Will remove it while squashing.]


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37557222
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    --- End diff --
    
    ??? Why don't you simple create the new ContinuousHistogram first. Directly operate on the data field and then assure that everything is merged correctly?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556594
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    --- End diff --
    
    Yeah. That should 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527271
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    --- End diff --
    
    Doesn't the 2. belong to the **Discrete Histograms** 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137592
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    if (data.get(c).isEmpty) {
    +      require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +      data.put(c, 1)
    +    } else {
    +      data.update(c, data.get(c).get + 1)
    +    }
    --- End diff --
    
    If use pattern matching for `data.get(c)`, we can reduce calling `data.get(c)`.
    
    For example:
    
    ```scala
    data.get(c) match {
      case None =>
        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
        data.put(c, 1)
      case Some(v) => data.update(c, v + 1)
    }
    ```


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37526853
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    --- End diff --
    
    Curly braces are missing here to make `x: ...` a set. 


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137866
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/FieldStats.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.statistics
    +
    +import org.apache.flink.ml.statistics.FieldType._
    +
    +import scala.collection.mutable
    +
    +/** Class to represent Field statistics.
    +  *
    +  * =Parameters=
    +  * -[[fieldType]]:
    +  *   Type of this field, [[DISCRETE]] or [[CONTINUOUS]]
    +  *
    +  * For [[DISCRETE]] fields, [[entropy]], [[gini]] and [[categoryCounts]] are provided.
    +  * For [[CONTINUOUS]] fields, [[min]], [[max]], [[mean]] and [[variance]] are provided.
    +  *
    +  */
    +class FieldStats(val fieldType: FieldType) extends Serializable {
    +  // field parameters
    +  private [statistics] var _min: Double = _
    +  private [statistics] var _max: Double = _
    +  private [statistics] var _mean: Double = _
    +  private [statistics] var _variance: Double = _
    +  private [statistics] var _counts: mutable.HashMap[Double,Int] = _
    +
    +  //-------------------- Access methods ----------------------------//
    +  def min: Double = {
    +    exception(DISCRETE)
    +    _min
    +  }
    +
    +  def max: Double = {
    +    exception(DISCRETE)
    +    _max
    +  }
    +
    +  def mean: Double = {
    +    exception(DISCRETE)
    +    _mean
    +  }
    +
    +  def variance: Double = {
    +    exception(DISCRETE)
    +    _variance
    +  }
    +
    +  def categoryCounts: mutable.HashMap[Double,Int] = {
    +    exception(CONTINUOUS)
    +    _counts
    +  }
    +
    +  /**
    +   * Returns the entropy value for this [[DISCRETE]] field.
    +   */
    +  def entropy: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    3.322 * _counts.iterator.map(x => - (x._2 / total) * Math.log10(x._2 / total)).sum
    +  }
    +
    +  /**
    +   * Returns the Gini impurity for this [[DISCRETE]] field.
    +   */
    +  def gini: Double = {
    +    exception(CONTINUOUS)
    +    val total: Double = _counts.valuesIterator.sum
    +    1 - _counts.iterator.map(x => (x._2 * x._2)).sum / (total * total)
    --- End diff --
    
    Unnecessary parentheses in `(x._2 * x._2)`.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132131366
  
    I would vote for returning a continuous histogram as the default and perhaps provide a discrete one as private to the ml package. 


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132118352
  
    What about having two different functions? One for Discrete and one for continuous? 
    Or perhaps just one for the Continuous as that is more likely to be used.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115202017
  
    For the moment, I think it's best to place it under `org.apache.flink.ml.density`, for example.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527495
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    --- End diff --
    
    What's the meaning of the bins?


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556791
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    +    val size: Int = bins
    +    if (size == 0 || p < getValue(0)) {
    +      return 0
    +    }
    +    for (i <- 0 to size - 2) {
    +      if (p >= getValue(i) && p < getValue(i + 1)) {
    +        return i + 1
    +      }
    +    }
    +    size
    +  }
    +
    +  /** Merges the closest two elements in the histogram
    +    * If we're over capacity, merge without thought
    +    * Otherwise, only merge when two elements have almost equal values. We're talking doubles, so
    +    * no exact equalities
    +    *
    +    * @return Whether we succeeded in merging any two elements
    +    */
    +  private def mergeElements(): Boolean = {
    --- End diff --
    
    You're contradicting yourself within two sentences. It happens when you load the data. And this happens when you merge two histograms. Thus it happens quite some time. What about a priority queue with an additional set for invalidated entries?


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33356477
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,337 @@
    +/*
    + * 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.math
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  */
    +case class ContinuousHistogram(
    +    capacity: Int,
    +    min: Double,
    +    max: Double)
    +  extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else if (getValue(i) <= temp.getValue(j)) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else return getValue(i - 1) +
    +          (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +      } else currSum = tmpSum
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[math] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.size > 0) {
    --- End diff --
    
    Replace with `if (data.nonEmpty)`


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37531273
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    --- End diff --
    
    This will throw an `IllegalArgumentException`. I think it's a bit harsh to throw an exception when the histogram is empty.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537538
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match {
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        val finalSize = if (B > 0) B else finalMap.size
    +        val ret = new DiscreteHistogram(finalSize)
    +        ret.loadData(finalMap.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a discrete histogram is allowed to be merged with a " +
    +          "discrete histogram")
    +    }
    +  }
    +
    +  /** Number of elements in category c
    +    *
    +    * @return Number of points in category c
    +    */
    +  def count(c: Double): Int = {
    +    data.get(c) match {
    +      case None => 0
    +      case Some(value) => value
    +    }
    +  }
    +
    +  /** Returns the total number of elements in the histogram
    +    *
    +    * @return total number of elements added so far
    +    */
    +  override def total: Int = data.values.sum
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    --- End diff --
    
    remove 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37531763
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    --- End diff --
    
    There is no valid return value here if the histogram is empty.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33151075
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.math
    +
    +import java.lang.Double.MAX_VALUE
    +import java.util
    --- End diff --
    
    Is using only Scala library more efficient? It does make the code coherent (is that the right word?) to use just one library: Java or Scala. Let me know. Should be easy enough to change though.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33050123
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/OnlineHistogram.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.math
    +
    +/** Base trait for an Online Histogram
    +  *
    +  */
    +trait OnlineHistogram {
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  def bins: Int
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  def add(p: Double): Unit
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  def merge(h: OnlineHistogram, B: Int): OnlineHistogram
    +
    +  /** Returns the qth quantile of the histogram
    +    * Should fail for a discrete version
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double
    --- End diff --
    
    I was not sure about this. But yeah that would make more sense. I made these functions part of the trait itself for the ease of use later on. I will update 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534762
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    +    val size: Int = bins
    +    if (size == 0 || p < getValue(0)) {
    +      return 0
    +    }
    +    for (i <- 0 to size - 2) {
    +      if (p >= getValue(i) && p < getValue(i + 1)) {
    +        return i + 1
    +      }
    +    }
    +    size
    +  }
    +
    +  /** Merges the closest two elements in the histogram
    +    * If we're over capacity, merge without thought
    +    * Otherwise, only merge when two elements have almost equal values. We're talking doubles, so
    +    * no exact equalities
    +    *
    +    * @return Whether we succeeded in merging any two elements
    +    */
    +  private def mergeElements(): Boolean = {
    +    val size: Int = bins
    +    var minDiffIndex: Int = -1
    +    var minDiff: Double = MaxValue
    +    for (i <- 0 to size - 2) {
    +      val currDiff: Double = getValue(i + 1) - getValue(i)
    +      if (currDiff < minDiff) {
    +        minDiff = currDiff
    +        minDiffIndex = i
    +      }
    +    }
    --- End diff --
    
    insert line break 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37560144
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    --- End diff --
    
    Starting from the minimum valued bin, we keep going till we land between two bins(x and y) between whose counts our required value(s) lies.
    Since, we only need s - count(x) elements from between x and y, we construct a trapezoid, as in the figure.
    The area of the whole trapezoid corresponds to (m_x + m_y) / 2 elements.
    Area of trapezoid till s is a quadratic in s, and this can be transformed to number of elements.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33047408
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/CategoricalHistogram.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.math
    +
    +import java.util
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  */
    +class CategoricalHistogram(
    +    values: Array[Double],
    +    counts: Array[Int] = Array.ofDim[Int](0))
    +  extends OnlineHistogram
    +  with Serializable {
    +
    +  // sanity checks
    +  require(values.length > 0, "Number of classes should be a positive integer")
    +  require(counts.length == 0 || counts.length == values.length, "Counts should have the same " +
    +    "number" + " " + "of " + "entries as values")
    +  val data = new util.ArrayList[(Double, Int)](values.length)
    +  require(checkSanity, "All values in counts should be non-negative")
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    // search for the index where the value is equal to1 p
    +    val search = find(p)
    +    require(search >= 0, p + "is not present in the histogram")
    +    val currentVal = data.get(search)
    +    data.set(search, (currentVal._1, currentVal._2 + 1))
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): CategoricalHistogram = {
    +    require(h.isInstanceOf[CategoricalHistogram],
    +      "Only a continuous histogram is allowed to be merged with a continuous histogram")
    +    val h1 = h.asInstanceOf[CategoricalHistogram]
    --- End diff --
    
    Better to use type matching here and throw an exception if h is not a CategoricalHistogram


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115574532
  
    Hello Sachin,
    
    although we will for now postpone adding the histogram function to DataSetUtil, we should still have some external functionality for this PR.
    We discussed this a bit with Till and the best place to put the histogram function now is in the package.scala of the ml package.
    
    You will see that there already some pimp-my-class style functions there, like RichLabeledDataSet that has the writeAsLibSVM function.
    
    So you can do the same and have an implicit RichDoubleDataSet class that defines the histogram function for DataSet[Double]. Once we have that we can 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137658
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    if (data.get(c).isEmpty) {
    +      require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +      data.put(c, 1)
    +    } else {
    +      data.update(c, data.get(c).get + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): CategoricalHistogram = {
    +    h match {
    +      case h1: CategoricalHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          if (finalMap.get(x._1).isEmpty) {
    +            finalMap.put(x._1, x._2)
    +          } else {
    +            finalMap.update(x._1, x._2 + finalMap.get(x._1).get)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        val countBuffer = new mutable.ArrayBuffer[(Double, Int)]()
    +        finalMap.iterator.foreach(x => countBuffer += ((x._1, x._2)))
    +        var finalSize = finalMap.size
    +        if (B > 0) {
    +          finalSize = B
    +        }
    +        val ret = new CategoricalHistogram(finalSize)
    +        ret.loadData(countBuffer.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a categorical histogram is allowed to be merged with a " +
    +          "categorical histogram")
    +    }
    +  }
    +
    +  /** Number of elements in category c
    +    *
    +    * @return Number of points in category c
    +    */
    +  def count(c: Double): Int = {
    +    if (data.get(c).isEmpty) {
    +      return 0
    +    }
    +    data.get(c).get
    +  }
    +
    +  /** Returns the total number of elements in the histogram
    +    *
    +    * @return total number of elements added so far
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    data.valuesIterator.foreach(x => result += x)
    +    result
    +  }
    --- End diff --
    
    We can implement `total` method without mutable variable.
    
    ```scala
    override def total: Int = data.valuesIterator.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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#discussion_r38616759
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/accumulators/ContinuousHistogram.java ---
    @@ -0,0 +1,490 @@
    +/*
    + * 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.api.common.accumulators;
    +
    +import java.util.AbstractMap;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.PriorityQueue;
    +import java.util.Set;
    +import java.util.TreeMap;
    +
    +import static java.lang.Double.MAX_VALUE;
    +
    +/**
    + * A Histogram accumulator designed for Continuous valued data.
    + * It supports:
    + * -- {@link #quantile(double)}
    + * 		Computes a quantile of the data
    + * -- {@link #count(double)}
    + *		Computes number of items less than the given value in the data
    + * <p>
    + * A continuous histogram stores values in bins in sorted order and keeps their associated
    + * number of items. It is assumed that the items associated with every bin are scattered around
    + * it, half to the right and half to the left.
    + * <p>
    + * bin counters:  m_1    m_2    m_3    m_4    m_5    m_6
    + *                10     12     5      10     4      6
    + *                |  5   |  6   |  2.5 |  5   |  2   |
    + *             5  |  +   |  +   |   +  |  +   |  +   |  3
    + *                |  6   |  2.5 |   5  |  2   |  3   |
    + * - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    + * bin index:     1      2      3      4      5      6
    + * bin values:    v_1 <  v_2 <  v_3 <  v_4 <  v_5 <  v_6
    + * <p>
    + * The number of items between v_i and v_(i+1) is directly proportional to the area of
    + * trapezoid (v_i, v_(i+1), m_(i+1), m_i)
    + * <p>
    + * Adapted from Ben-Haim and Yom-Tov's
    + * <a href = "http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf">Streaming Decision Tree Algorithm's histogram</a>
    + */
    +public class ContinuousHistogram extends Histogram {
    +
    +	private int bin;
    +	private double lower;
    +	private double upper;
    +	private PriorityQueue<KeyDiff> diffQueue;
    +	private HashMap<Double, KeyProps> keyUpdateTimes;
    +	private long timestamp;
    +
    +	/**
    +	 * Creates a new Continuous histogram with the given number of bins
    +	 * Bins represents the number of values the histogram stores to approximate the continuous
    +	 * data set. The higher this value, the more we move towards an exact representation of data.
    +	 *
    +	 * @param bin Number of bins in the histogram
    +	 */
    +	public ContinuousHistogram(int bin) {
    +		if (bin <= 0) {
    +			throw new IllegalArgumentException("Number of bins must be greater than zero");
    +		}
    +		this.bin = bin;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue = new PriorityQueue<>();
    +		keyUpdateTimes = new HashMap<>();
    +		timestamp = 0;
    +	}
    +
    +	@Override
    +	public void resetLocal() {
    +		super.resetLocal();
    +		this.lower = MAX_VALUE;
    +		this.upper = -MAX_VALUE;
    +		this.diffQueue.clear();
    +		this.keyUpdateTimes.clear();
    +	}
    +
    +	/**
    +	 * Merges the given other histogram into this histogram, with the number of bins in the
    +	 * merged histogram being {@code numBins}.
    +	 *
    +	 * @param other   Histogram to be merged
    +	 * @param numBins Bins in the merged histogram
    +	 */
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other, int numBins) {
    +		bin = numBins;
    +		super.merge(other);
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "Continuous " + super.toString();
    +	}
    +
    +	@Override
    +	public Accumulator<Double, TreeMap<Double, Integer>> clone() {
    +		ContinuousHistogram result = new ContinuousHistogram(bin);
    +		result.treeMap = new TreeMap<>(treeMap);
    +		result.counter = counter;
    +		result.lower = lower;
    +		result.upper = upper;
    +		// initialize all differences and key update times for the new histogram
    +		result.computeDiffs();
    +		return result;
    +	}
    +
    +	@Override
    +	void add(double value, int count) {
    +		addValue(value, count);
    +		if (getSize() > bin) {
    +			mergeBins();
    +		}
    +	}
    +
    +	@Override
    +	void fill(Set<Map.Entry<Double, Integer>> entries) {
    +		for (Map.Entry<Double, Integer> entry : entries) {
    +			if (entry.getValue() <= 0) {
    +				throw new IllegalArgumentException("Negative counters are not allowed: " + entry);
    +			}
    +		}
    +
    +		for (Map.Entry<Double, Integer> entry : entries) {
    +			addValue(entry.getKey(), entry.getValue());
    +		}
    +
    +		while (getSize() > bin) {
    +			mergeBins();
    +		}
    +	}
    +
    +	/**
    +	 * Adds a new value to the histogram along with an associated count.
    +	 *
    +	 * @param value Value to be added
    +	 * @param count Associated count to this value
    +	 */
    +	private void addValue(double value, int count) {
    +		if (value < lower) {
    +			lower = value;
    +		}
    +		if (value > upper) {
    +			upper = value;
    +		}
    +
    +		// Add to the map.
    +		counter += count;
    +		Integer current = treeMap.get(value);
    +		Integer newValue = (current != null ? current : 0) + count;
    +		this.treeMap.put(value, newValue);
    --- End diff --
    
    Please remove `this` to increase uniformity.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115207592
  
    You don't do it. I think it's best at the moment to only make the histograms available within the ml package. Everyone who wants to use them, can then add `flink-ml` as a dependency.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37551595
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    --- End diff --
    
    Suppose, the count at index i - 1 < required sum( = s) and count at index (i) > required sum.
    Then, the required value, say b, lies between v_(i - 1) and v_i, and the area of trapezoid uptil b must be (s - count(v_(i - 1))).
    This leads to solving a quadratic in b.



---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115653886
  
    Okay. Finished. The function definition is
    ``` scala
    createHistogram(bins: Int) : DataSet[OnlineHistogram] = {
    ...
    }
    ```


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137759354
  
    @sachingoel0101 That sounds like a good place for the docs, that `zip_elements_guide.mb` is linked from the DataSet Transformations doc, so replacing that link to a more general "DataSet Utilities" doc would make sense.
    
    I'm looking into the wrapper for Scala, it's a bit more convoluted than I thought because of scala.Double being a Java primitive double so the conversion is not straightforward.
    
    



---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114896540
  
    >The idea is to emulate a discrete probability distribution, which it does. For the continuous case, we're emulating a continuous probability distribution, which means we can effectively add any value we want.
    
    OK so we don't provide a online discrete histogram in the general sense, but rather one that specifically aimed at modeling a discrete probability distribution. That does kind of limit the applicability of the code to the specific use case of the decision tree, doesn't it? Do you think it's possible to generalize the discrete histogram code without breaking the decision tree functionality? 


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37532289
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    --- End diff --
    
    return statements are not really scalaesque


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114899521
  
    Changing the current discrete histogram implementation would not break the decision tree functionality. Although I might have to review the code for any potential errors that might creep in. 
    
    For the utility function, it doesn't really make a difference. We could simply have two functions which would treat the data as continuous or discrete.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132146208
  
    Sounds good. 


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37554978
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    **Discrete Histograms**: These histograms are formed on a data set `X:DataSet[Double]`
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A discrete histogram can be formed by calling `MLUtils.createDiscreteHistogram(X)`.
    --- End diff --
    
    Hmm, why do we have discrete histograms? Are they 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144130
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,100 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. <strong>Continuous Histograms</strong>: These histograms are formed on a data set `X:
    --- End diff --
    
    `<strong>` tag can be replaced by `**`. `**Continuous Histograms**` is same as `<strong>Continuous Histograms</strong>`.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37531123
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    --- End diff --
    
    *being used(, whereas in the base class it's written *being utilized*. Maybe we can harmonize that. 


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37554555
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    --- End diff --
    
    I understand what you want to say, but I think it's not well formulated. IMO it's better to clearly define what `sum(s)` or better what `count(s)` means. E.g. "The value sum(s) represents the number of elements in X whose value is less than s" as you've said. But the rest is not 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534224
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    --- End diff --
    
    remove *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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529318
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    --- End diff --
    
    Sure. :)


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37528219
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    +      var statistics: (Double, Double) = _
    +
    +      override def open(configuration: Configuration): Unit = {
    +        statistics = getRuntimeContext.getBroadcastVariable(HISTOGRAM_STATS).get(0)
    +        val minimum = statistics._1
    +        val maximum = statistics._2
    +        statistics = (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    --- End diff --
    
    You already calculate this value in line 138 and then broadcast this value to the `RichMapPartitionFunction`. 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529224
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    +      var statistics: (Double, Double) = _
    +
    +      override def open(configuration: Configuration): Unit = {
    +        statistics = getRuntimeContext.getBroadcastVariable(HISTOGRAM_STATS).get(0)
    +        val minimum = statistics._1
    +        val maximum = statistics._2
    +        statistics = (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +      }
    +
    +      override def mapPartition(
    +          values: java.lang.Iterable[Double],
    +          out: Collector[ContinuousHistogram])
    +        : Unit = {
    +        val localHistogram = new ContinuousHistogram(bins, statistics._1, statistics._2)
    +        val iterator = values.iterator()
    +        while (iterator.hasNext) {
    +          localHistogram.add(iterator.next())
    +        }
    +        out.collect(localHistogram)
    +      }
    +    })
    +      .withBroadcastSet(stats, HISTOGRAM_STATS)
    +      .reduce((x, y) => x.merge(y, bins))
    +  }
    +
    +  /** Create a [[DiscreteHistogram]] from the input data
    +    *
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[DiscreteHistogram]] over the data
    +    */
    +  def createDiscreteHistogram(data: DataSet[Double]): DataSet[DiscreteHistogram] = {
    +    data.mapPartition(new RichMapPartitionFunction[Double, DiscreteHistogram] {
    --- End diff --
    
    Yes, but that would require creating a Histogram object for every element. This way we can work with just one object per partition.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37539017
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match {
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    --- End diff --
    
    Providing the capacity here is a strict check that both histograms, when merged, don't overshoot somehow over the number of classes we actually know. 
    There is a option to turn this off, by specifying B to be 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556789
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    --- End diff --
    
    Will fix 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37536461
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    --- End diff --
    
    backticks are no valid scaladocs syntax as far as I know


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556169
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    --- End diff --
    
    Rhetorical again. Look at the function you're describing. It's called count and you write something about probability. How does this fit? And I know that it can be seen as the scaled probability. But if you describe things as a transformation of the actual thing you want to describe, then you will only confuse people. E.g. you could also apply the logarithm to the probabiliy value and then talk about a value in the range [-inf, 0]. Isn't this a little bit confusing?


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132183402
  
    Is there the valid reason to have multiple names ('discrete histogram', 'categorical histogram')? I think that to avoid confusion, we need to unify the names.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144073
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +120,31 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[DataSet]] of [[OnlineHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required. Zero for [[CategoricalHistogram]]
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[DataSet]] of [[OnlineHistogram]]
    +    */
    +  private [ml] def createHistogram(data: DataSet[Double], bins: Int): DataSet[OnlineHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max){
    +      (minimum,maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +    val ret: DataSet[OnlineHistogram] = data.mapWithBcVariable(stats){ (x, statistics) =>
    +      if(bins > 0){
    +        val h = new ContinuousHistogram(1, statistics._1, statistics._2)
    +        h.loadData(Array((x, 1)))
    +        h
    +      } else{
    --- End diff --
    
    Need a space between `else` and `{`.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33143611
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.math
    +
    +import java.lang.Double.MAX_VALUE
    +import java.util
    --- End diff --
    
    Why use Java's ArrayList here instead of Scala's ArrayBuffer 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-149494676
  
    Not yet. Just finished my current task. Will take a look now.
    
    On Tue, Oct 20, 2015 at 11:25 AM, Sachin Goel <no...@github.com>
    wrote:
    
    > @tillrohrmann <https://github.com/tillrohrmann> , any updates regarding
    > this?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/861#issuecomment-149491074>.
    >



---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-131514958
  
    I found some points to improve and added line notes. :)


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132264800
  
    Haha. It's no problem. I've updated the documentation 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132188989
  
    @chiwanpark , modified the names.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556660
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    --- End diff --
    
    Ah. Apologies. I was drawing the analogy from Java.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37143932
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/FieldStats.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.statistics
    +
    +import org.apache.flink.ml.statistics.FieldType._
    +
    +import scala.collection.mutable
    +
    +/** Class to represent Field statistics.
    +  *
    +  * =Parameters=
    +  * -[[fieldType]]:
    +  *   Type of this field, [[DISCRETE]] or [[CONTINUOUS]]
    +  *
    +  * For [[DISCRETE]] fields, [[entropy]], [[gini]] and [[categoryCounts]] are provided.
    +  * For [[CONTINUOUS]] fields, [[min]], [[max]], [[mean]] and [[variance]] are provided.
    +  *
    +  */
    +class FieldStats(val fieldType: FieldType) extends Serializable {
    +  // field parameters
    +  private [statistics] var _min: Double = _
    +  private [statistics] var _max: Double = _
    +  private [statistics] var _mean: Double = _
    +  private [statistics] var _variance: Double = _
    +  private [statistics] var _counts: mutable.HashMap[Double,Int] = _
    +
    --- End diff --
    
    I meant that the space between `private` and `[statistics]` should be removed. It is just cosmetic issue but to keep unified code style, we need fix 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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144182
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match{
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): CategoricalHistogram = {
    +    h match {
    +      case h1: CategoricalHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match{
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        var finalSize = finalMap.size
    +        if (B > 0) {
    +          finalSize = B
    +        }
    --- End diff --
    
    We can change `finalSize` from mutable to immutable.
    
    ```scala
    val finalSize = if (B > 0) B else finalMap.size
    ```


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137936053
  
    I agree, let's not break the API in this PR. We can create an issue and have a small discussion on the list about the change, and if the community agrees we can break the API in another PR specifically for that issue.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33359906
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/MLUtilsSuite.scala ---
    @@ -109,4 +109,41 @@ class MLUtilsSuite extends FlatSpec with Matchers with FlinkTestBase {
     
         tempFile.delete()
       }
    +
    +  it should "create a discrete histogram" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    val discreteData = Seq(1.0, 2.0, 3.0, 5.0, 1.0, 7.0, 9.0, 1.0, 0.0, 1.0, 4.0, 6.0, 7.0, 9.0,
    +      4.0, 3.0, 1.0, 4.0, 6.0, 8.0, 4.0, 3.0, 6.0, 8.0, 4.0, 3.0, 6.0, 8.0, 9.0, 7.0, 8.0, 2.0, 3.0,
    +      6.0, 0.0)
    +    val h = env.fromCollection(discreteData)
    +      .createHistogram(0)
    +      .collect().toArray.apply(0)
    +      .asInstanceOf[CategoricalHistogram]
    +
    +    h.count(0) should equal(discreteData.filter(x => x == 0).size)
    +    h.count(1) should equal(discreteData.filter(x => x == 1).size)
    +    h.count(2) should equal(discreteData.filter(x => x == 2).size)
    +    h.count(3) should equal(discreteData.filter(x => x == 3).size)
    +    h.count(4) should equal(discreteData.filter(x => x == 4).size)
    +    h.count(5) should equal(discreteData.filter(x => x == 5).size)
    +    h.count(6) should equal(discreteData.filter(x => x == 6).size)
    +    h.count(7) should equal(discreteData.filter(x => x == 7).size)
    +    h.count(8) should equal(discreteData.filter(x => x == 8).size)
    +    h.count(9) should equal(discreteData.filter(x => x == 9).size)
    +  }
    +
    +  it should "create a continuous histogram" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    val discreteData = Seq(1.0, 2.0, 3.0, 5.0, 1.0, 7.0, 9.0, 1.0, 0.0, 1.0, 4.0, 6.0, 7.0, 9.0,
    +      4.0, 3.0, 1.0, 4.0, 6.0, 8.0, 4.0, 3.0, 6.0, 8.0, 4.0, 3.0, 6.0, 8.0, 9.0, 7.0, 8.0, 2.0, 3.0,
    +      6.0, 0.0)
    +    val h = env.fromCollection(discreteData)
    +      .createHistogram(5)
    +      .collect().toArray.apply(0)
    +      .asInstanceOf[ContinuousHistogram]
    --- End diff --
    
    Ideally we would like to avoid having to cast, and just return the correct type, depending on the provided arguments. Or always return a ContinuousHistogram as I originally suggested.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-133002882
  
    @tillrohrmann Sure. :)


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37141028
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/FieldStats.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.statistics
    +
    +import org.apache.flink.ml.statistics.FieldType._
    +
    +import scala.collection.mutable
    +
    +/** Class to represent Field statistics.
    +  *
    +  * =Parameters=
    +  * -[[fieldType]]:
    +  *   Type of this field, [[DISCRETE]] or [[CONTINUOUS]]
    +  *
    +  * For [[DISCRETE]] fields, [[entropy]], [[gini]] and [[categoryCounts]] are provided.
    +  * For [[CONTINUOUS]] fields, [[min]], [[max]], [[mean]] and [[variance]] are provided.
    +  *
    +  */
    +class FieldStats(val fieldType: FieldType) extends Serializable {
    +  // field parameters
    +  private [statistics] var _min: Double = _
    +  private [statistics] var _max: Double = _
    +  private [statistics] var _mean: Double = _
    +  private [statistics] var _variance: Double = _
    +  private [statistics] var _counts: mutable.HashMap[Double,Int] = _
    +
    --- End diff --
    
    Could you clarify this point? They're already package private fields.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-131520764
  
    I just found that `Predef.assume` could be replaced by `assume`. Sorry for wrong guidance. Could you change `Predef.assume` to `assume`?


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144195
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/CategoricalHistogram.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  *   Number of categories in the histogram
    +  */
    +case class CategoricalHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match{
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): CategoricalHistogram = {
    +    h match {
    +      case h1: CategoricalHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match{
    --- End diff --
    
    A space is needed between `match` and `{`.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-141471679
  
    Rebased to reflect the changes in the scala utility functions.
    Travis failure on an unrelated error. Reported at jira id 2700.
    
    This has already undergone three reviews, and been open for almost four months (as part of #710 too). If this functionality is not required, I can close the PR; that'd however automatically mean closing #710 too since this is essential to implementing a Decision Tree. I'd prefer not to though as I've spent a lot of time optimizing this.
    
    @chiwanpark @tillrohrmann 


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115210476
  
    Okay. So I guess we can leave adding a createHistogram function to DataSetUtils for now [It would also require utilizing the FlinkMLTools.block for an efficient implementation]. Pending that, this PR is ready to merge then. Please have a look for any other modifications that are needed.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115179641
  
    @thvasilo, how should I link this PR to #832 ? Should I fork Andra's repo and create a PR 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114901089
  
    > For the utility function, it doesn't really make a difference. We could simply have two functions which would treat the data as continuous or discrete.
    
    I would prefer to keep this simple and only have one histogram function. We can add an argument isDiscrete that defaults to false, and it follows the correct codepath depending on the provided value.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137912454
  
    I object to break API in this PR. I think that this PR should cover histogram for `DataSet<Double>` only. That's why we split this PR from #1032. About documentation, adding a document file such as `building_histogram.md` first and merging it later would be good.
    
    For Scala API, the current implementation looks good for me.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137399688
  
    Hey @chiwanpark , travis passes. Let me know if I should squash the commits. I'm only keeping them in case we need to go back to the Scala implementation. [which I really don't see the point of though.]


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33143726
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.math
    +
    +import java.lang.Double.MAX_VALUE
    --- End diff --
    
    Use Double.MaxValue instead?


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114901644
  
    Okay. Sure.
    I will update the make the Discrete version online first. 
    Should I try to explicitly use Scala library data structures instead of Java?


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114895657
  
    Adding a Utility method does certainly make sense. User will be supposed to provide an argument depicting whether the values in DataSet[Double] are continuous or discrete.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537083
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    +    val size: Int = bins
    +    if (size == 0 || p < getValue(0)) {
    +      return 0
    +    }
    +    for (i <- 0 to size - 2) {
    +      if (p >= getValue(i) && p < getValue(i + 1)) {
    +        return i + 1
    +      }
    +    }
    +    size
    +  }
    +
    +  /** Merges the closest two elements in the histogram
    +    * If we're over capacity, merge without thought
    +    * Otherwise, only merge when two elements have almost equal values. We're talking doubles, so
    +    * no exact equalities
    +    *
    +    * @return Whether we succeeded in merging any two elements
    +    */
    +  private def mergeElements(): Boolean = {
    --- End diff --
    
    Capacity won't ever be exceeded by more than 1. 
    That will only happen at the time of loading data, or when two histograms are merged. I had thought about it a lot, but couldn't figure out a proper data structure to accomplish this. 
    Since we only merge the least separated values, the differences themselves vary and the index at which they're present vary also. 


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537596
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match {
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        val finalSize = if (B > 0) B else finalMap.size
    +        val ret = new DiscreteHistogram(finalSize)
    +        ret.loadData(finalMap.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a discrete histogram is allowed to be merged with a " +
    +          "discrete histogram")
    +    }
    +  }
    +
    +  /** Number of elements in category c
    +    *
    +    * @return Number of points in category c
    +    */
    +  def count(c: Double): Int = {
    +    data.get(c) match {
    +      case None => 0
    +      case Some(value) => value
    +    }
    +  }
    +
    +  /** Returns the total number of elements in the histogram
    +    *
    +    * @return total number of elements added so far
    +    */
    +  override def total: Int = data.values.sum
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (category,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    require(counts.size <= bins, "Insufficient capacity. Failed to initialize")
    +    counts.iterator.foreach(valCount => {
    --- End diff --
    
    you don't need to call `iterator`


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37528613
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    +      var statistics: (Double, Double) = _
    +
    +      override def open(configuration: Configuration): Unit = {
    +        statistics = getRuntimeContext.getBroadcastVariable(HISTOGRAM_STATS).get(0)
    +        val minimum = statistics._1
    +        val maximum = statistics._2
    +        statistics = (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    --- End diff --
    
    No. The one in 148 shouldn't be here. This is actually intended to be added in the next commit with Column-wise statistics. I'll remove 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-141926745
  
    Okay. Great.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132512002
  
    Okay. Done. 
    I will modify the `createContinuousHistogram` to use `DataStats` in #1032, since that is going to be merged after this anyway. 


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137856
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  *   Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  *   Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  *   Upper limit on the elements
    +  */
    +case class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += (data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    counts.iterator.foreach(x => data += ((x._1, x._2)))
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Returns the lower limit on values of the histogram
    +    *
    +    * @return lower limit on values
    +    */
    +  private def lower: Double = {
    +    min
    +  }
    +
    +  /** Returns the upper limit on values of the histogram
    +    *
    +    * @return upper limit on values
    +    */
    +  private def upper: Double = {
    +    max
    +  }
    --- End diff --
    
    Because `min` and `max` is determined in initializing, `lower` and `upper` could be declared as immutable variable.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144173
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,100 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. <strong>Continuous Histograms</strong>: These histograms are formed on a data set `X:
    +   DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, <i>scaled</i> probability].
    +   <br>
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    <strong>Categorical Histograms</strong>: These histograms are formed on a data set `X:DataSet[Double]` 
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A categorical histogram can be formed by calling `X.createHistogram(0)`.
    +
    +### Data Statistics
    +
    + The `dataStats` function operates on a data set `X: DataSet[Vector]` and returns column-wise
    + statistics for `X`. Every field of `X` is allowed to be defined as either <i>discrete</i> or
    --- End diff --
    
    `<i>` tag


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37539214
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    --- End diff --
    
    Inefficient. You first create a list which you then convert to an array only to add it to another list 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137690196
  
    Looks good to merge except some minor issues. If there is no objection for this PR, I'll merge this in tomorrow.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37556758
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    --- End diff --
    
    Umm, writing an `if` statement inside a loop this way will simply return from the function call? Then I guess this will 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137381508
  
    I'm inclined to preventing base class for the histogram. Decision Tree will be implemented in Scala and we can use pattern matching (case-match statement) for solving 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#discussion_r38735587
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/accumulators/ContinuousHistogram.java ---
    @@ -0,0 +1,534 @@
    +/*
    + * 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.api.common.accumulators;
    +
    +import java.util.AbstractMap;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.PriorityQueue;
    +import java.util.Set;
    +import java.util.TreeMap;
    +
    +import static java.lang.Double.MAX_VALUE;
    +
    +/**
    + * A Histogram accumulator designed for Continuous valued data.
    + * It supports:
    + * -- {@link #quantile(double)}
    + * 		Computes a quantile of the data
    + * -- {@link #count(double)}
    + *		Computes number of items less than the given value in the data
    + * <p>
    + * A continuous histogram stores values in bins in sorted order and keeps their associated
    + * number of items. It is assumed that the items associated with every bin are scattered around
    + * it, half to the right and half to the left.
    + * <p>
    + * bin counters:  m_1    m_2    m_3    m_4    m_5    m_6
    + *                10     12     5      10     4      6
    + *                |  5   |  6   |  2.5 |  5   |  2   |
    + *             5  |  +   |  +   |   +  |  +   |  +   |  3
    + *                |  6   |  2.5 |   5  |  2   |  3   |
    + * - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    + * bin index:     1      2      3      4      5      6
    + * bin values:    v_1 <  v_2 <  v_3 <  v_4 <  v_5 <  v_6
    + * <p>
    + * The number of items between v_i and v_(i+1) is directly proportional to the area of
    + * trapezoid (v_i, v_(i+1), m_(i+1), m_i)
    + * <p>
    + * Adapted from Ben-Haim and Yom-Tov's
    + * <a href = "http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf">Streaming Decision Tree Algorithm's histogram</a>
    + */
    +public class ContinuousHistogram implements Accumulator<Double, TreeMap<Double, Integer>> {
    +
    +	protected TreeMap<Double, Integer> treeMap = new TreeMap<Double, Integer>();
    +
    +	protected long counter = 0;
    +
    +	private int bin;
    +
    +	private double lower;
    +
    +	private double upper;
    +
    +	private PriorityQueue<KeyDiff> diffQueue;
    +
    +	private HashMap<Double, KeyProps> keyUpdateTimes;
    +
    +	private long timestamp;
    +
    +	/**
    +	 * Creates a new Continuous histogram with the given number of bins
    +	 * Bins represents the number of values the histogram stores to approximate the continuous
    +	 * data set. The higher this value, the more we move towards an exact representation of data.
    +	 *
    +	 * @param numBins Number of bins in the histogram
    +	 */
    +	public ContinuousHistogram(int numBins) {
    +		if (numBins <= 0) {
    +			throw new IllegalArgumentException("Number of bins must be greater than zero");
    +		}
    +		bin = numBins;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue = new PriorityQueue<>();
    +		keyUpdateTimes = new HashMap<>();
    +		timestamp = 0;
    +	}
    +
    +	/**
    +	 * Consider using {@link #add(double)} for primitive double values to get better performance.
    +	 */
    +	@Override
    +	public void add(Double value) {
    +		add(value, 1);
    +	}
    +
    +	public void add(double value) {
    +		add(value, 1);
    +	}
    +
    +	@Override
    +	public TreeMap<Double, Integer> getLocalValue() {
    +		return this.treeMap;
    +	}
    +
    +	/**
    +	 * Get the total number of items added to this histogram.
    +	 * This is preserved across merge operations.
    +	 *
    +	 * @return Total number of items added to the histogram
    +	 */
    +	public long getTotal() {
    +		return counter;
    +	}
    +
    +	/**
    +	 * Get the current size of the {@link #treeMap}
    +	 *
    +	 * @return Size of the {@link #treeMap}
    +	 */
    +	public int getSize() {
    +		return treeMap.size();
    +	}
    +
    +	@Override
    +	public void resetLocal() {
    +		treeMap.clear();
    +		counter = 0;
    +		lower = MAX_VALUE;
    +		upper = -MAX_VALUE;
    +		diffQueue.clear();
    +		keyUpdateTimes.clear();
    +	}
    +
    +	@Override
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other) {
    +		fill(other.getLocalValue().entrySet());
    +	}
    +
    +	/**
    +	 * Merges the given other histogram into this histogram, with the number of bins in the
    +	 * merged histogram being {@code numBins}.
    +	 *
    +	 * @param other   Histogram to be merged
    +	 * @param numBins Bins in the merged histogram
    +	 */
    +	public void merge(Accumulator<Double, TreeMap<Double, Integer>> other, int numBins) {
    +		bin = numBins;
    +		merge(other);
    +	}
    +
    +	@Override
    +	public Accumulator<Double, TreeMap<Double, Integer>> clone() {
    +		ContinuousHistogram result = new ContinuousHistogram(bin);
    +		result.treeMap = new TreeMap<>(treeMap);
    +		result.counter = counter;
    +		result.lower = lower;
    +		result.upper = upper;
    +		// initialize all differences and key update times for the new histogram
    +		result.computeDiffs();
    +		return result;
    +	}
    +
    +	void add(double value, int count) {
    --- End diff --
    
    Why this method is protected?


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33046983
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/OnlineHistogram.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.math
    +
    +/** Base trait for an Online Histogram
    +  *
    +  */
    +trait OnlineHistogram {
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  def bins: Int
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  def add(p: Double): Unit
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  def merge(h: OnlineHistogram, B: Int): OnlineHistogram
    +
    +  /** Returns the qth quantile of the histogram
    +    * Should fail for a discrete version
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double
    --- End diff --
    
    About quantile and count, why not make them functions of the respective implementing classes? I don't think we should provide access to functions in the base class that just fail in certain implementations.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-136791499
  
    Okay. Sure. :) No problem.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#issuecomment-133099053
  
    Lemme optimize things and re-write the docs. I'll push a patch tomorrow. 
    Sorry about 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37528968
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    +      var statistics: (Double, Double) = _
    +
    +      override def open(configuration: Configuration): Unit = {
    +        statistics = getRuntimeContext.getBroadcastVariable(HISTOGRAM_STATS).get(0)
    +        val minimum = statistics._1
    +        val maximum = statistics._2
    +        statistics = (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +      }
    +
    +      override def mapPartition(
    +          values: java.lang.Iterable[Double],
    +          out: Collector[ContinuousHistogram])
    +        : Unit = {
    +        val localHistogram = new ContinuousHistogram(bins, statistics._1, statistics._2)
    +        val iterator = values.iterator()
    +        while (iterator.hasNext) {
    +          localHistogram.add(iterator.next())
    +        }
    +        out.collect(localHistogram)
    +      }
    +    })
    +      .withBroadcastSet(stats, HISTOGRAM_STATS)
    +      .reduce((x, y) => x.merge(y, bins))
    +  }
    +
    +  /** Create a [[DiscreteHistogram]] from the input data
    +    *
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[DiscreteHistogram]] over the data
    +    */
    +  def createDiscreteHistogram(data: DataSet[Double]): DataSet[DiscreteHistogram] = {
    +    data.mapPartition(new RichMapPartitionFunction[Double, DiscreteHistogram] {
    --- End diff --
    
    Why do you implement a `RichMapPartitionFunction` here? Isn't it easier to simply use an anonymous 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.
---

[GitHub] flink pull request: [FLINK-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37538024
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    --- End diff --
    
    Is there a reason why `DiscreteHistogram` and `ContinuousHistogram` both are subclasses of `OnlineHistogram` even though they don't share anything in common? I mean, they can't be merged nor do they share code via `OnlineHistogram`.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37531603
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    --- End diff --
    
    Misleading error message. There is no proceeding if it fails.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137570
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +120,30 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[DataSet]] of [[OnlineHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required. Zero for [[CategoricalHistogram]]
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[DataSet]] of [[OnlineHistogram]]
    +    */
    +  private [ml] def createHistogram(data: DataSet[Double], bins: Int): DataSet[OnlineHistogram] = {
    +    val min = data.map(x => x).reduce((x,y) => Math.min(x,y))
    +    val max = data.map(x => x).reduce((x,y) => Math.max(x,y))
    --- End diff --
    
    We don't need `map(x => x)` operation and need space between `x,` and `y`.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37527518
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,69 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and
    + `createDiscreteHistogram`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. **Continuous Histograms**: These histograms are formed on a data set `X: DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, *scaled* probability].
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    **Discrete Histograms**: These histograms are formed on a data set `X:DataSet[Double]`
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    --- End diff --
    
    html tags should be replaced by markdown syntax


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132116059
  
    I think that we can merge this PR after we decide the return type of `createHistogram` method. Any other points seem okay.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37547122
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    --- End diff --
    
    ![Image](https://lh5.googleusercontent.com/UNhh0eQOJ2ZwD1JOGAJZW1j_Mgn_3SFfmsnRqR7IZ6282dYuu6NzPhb7lRXY6AsDMILXsVtO=w1342-h523)
    
    I hope this image clears it up.
    Elements corresponding to the bin `x` are spread half to the left and half to the right of 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115199291
  
    Where should I place the Histogram implementations? Currently, they are in {{org.apache.flink.ml.math}}, but I can't import them from the flink-core where the DataSetUtils is located. Besides, since the purpose is to make the Histograms usable in general, they shouldn't be in the ml 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-121605463
  
    @thvasilo, @tillrohrmann, I'm still waiting for a decision on this. It would be impossible to work further on the decision tree PR until this is merged.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37528781
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    --- End diff --
    
    We could write this a little bit more scalaesque if we introduce a `mapPartitionWithBcVariable` 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.
---

[GitHub] flink pull request: [FLINK-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529217
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    --- End diff --
    
    Everywhere else the parameter is called bins but here it's capacity. I guess we should settle on one term.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33150869
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/CategoricalHistogram.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.math
    +
    +import java.util
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  */
    +class CategoricalHistogram(
    +    values: Array[Double],
    +    counts: Array[Int] = Array.ofDim[Int](0))
    +  extends OnlineHistogram
    +  with Serializable {
    +
    +  // sanity checks
    +  require(values.length > 0, "Number of classes should be a positive integer")
    +  require(counts.length == 0 || counts.length == values.length, "Counts should have the same " +
    +    "number" + " " + "of " + "entries as values")
    +  val data = new util.ArrayList[(Double, Int)](values.length)
    +  require(checkSanity, "All values in counts should be non-negative")
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    // search for the index where the value is equal to1 p
    +    val search = find(p)
    +    require(search >= 0, p + "is not present in the histogram")
    --- End diff --
    
    Yes. The categorical histogram is in fact not even mentioned in the paper. I saw fit to inherit them both from the same trait since effectively they represent statistics about a particular field, discrete or otherwise.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33356340
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,337 @@
    +/*
    + * 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.math
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  */
    +case class ContinuousHistogram(
    +    capacity: Int,
    +    min: Double,
    +    max: Double)
    +  extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else if (getValue(i) <= temp.getValue(j)) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else return getValue(i - 1) +
    +          (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +      } else currSum = tmpSum
    --- End diff --
    
    Surround else code with {} blocks, can remove return statements.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137952549
  
    @thvasilo , @chiwanpark I have added a wrapper for the scala utils class. Further, there are a few more methods:
    1. For Discrete histograms: *entropy* and *gini*
    2. For Continuous histograms: *min*, *max*, *mean*, *variance*
    
    I've updated the documentation to reflect these additions.
    
    This is more or less complete now. We can modify the scala utils api in a separate PR. I will file a JIRA for that and open a PR. It just involves some copy-pasting of the existing code,
    
    Should be mergeable 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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137778976
  
    Okay. Instead of adding the Scala api wrappers and add to the complexity [we might have another import such as `org.apache.flink.api.scala.DataSetUtils.utilsToDoubleDataSet`], maybe we can add that  with the next PR then?



---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33356751
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,337 @@
    +/*
    + * 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.math
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  */
    +case class ContinuousHistogram(
    +    capacity: Int,
    +    min: Double,
    +    max: Double)
    +  extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else if (getValue(i) <= temp.getValue(j)) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else return getValue(i - 1) +
    +          (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +      } else currSum = tmpSum
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[math] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.size > 0) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    counts.iterator.foreach(x => data.+=((x._1, x._2)))
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[math] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Returns the lower limit on values of the histogram
    +    *
    +    * @return lower limit on values
    +    */
    +  private def lower: Double = {
    +    min
    +  }
    +
    +  /** Returns the upper limit on values of the histogram
    +    *
    +    * @return upper limit on values
    +    */
    +  private def upper: Double = {
    +    max
    +  }
    +
    +  /** Checks whether the arraylist provided is properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    --- End diff --
    
    This seems a bit dangerous. What are the termination guarantees of mergeElements? Can we avoid the while statement 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#discussion_r33358232
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/math/ContinuousHistogram.scala ---
    @@ -0,0 +1,337 @@
    +/*
    + * 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.math
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  */
    +case class ContinuousHistogram(
    +    capacity: Int,
    +    min: Double,
    +    max: Double)
    +  extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new instance
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else if (getValue(i) <= temp.getValue(j)) {
    +            mergeList.+=(data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList.+=((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else return getValue(i - 1) +
    +          (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +      } else currSum = tmpSum
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[math] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.size > 0) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    counts.iterator.foreach(x => data.+=((x._1, x._2)))
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[math] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Returns the lower limit on values of the histogram
    +    *
    +    * @return lower limit on values
    +    */
    +  private def lower: Double = {
    +    min
    +  }
    +
    +  /** Returns the upper limit on values of the histogram
    +    *
    +    * @return upper limit on values
    +    */
    +  private def upper: Double = {
    +    max
    +  }
    +
    +  /** Checks whether the arraylist provided is properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    --- End diff --
    
    The purpose of mergeElements is to keep merging elements till we're upto allowed capacity or there are elements left which are really close (within 1e-9 of each other). So, it'll definitely terminate when there's only one element left in the histogram. 


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137802768
  
    I have split this into two commits.
    @thvasilo, if there is need to file a jira, I'll update the second commit message to indicate that.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37555025
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/MLUtils.scala ---
    @@ -119,4 +123,65 @@ object MLUtils {
     
         stringRepresentation.writeAsText(filePath)
       }
    +
    +  /** Create a [[ContinuousHistogram]] from the input data
    +    *
    +    * @param bins Number of bins required
    +    * @param data input [[DataSet]] of [[Double]]
    +    * @return [[ContinuousHistogram]] over the data
    +    */
    +  def createContinuousHistogram(data: DataSet[Double], bins: Int): DataSet[ContinuousHistogram] = {
    +    val min = data.reduce((x, y) => Math.min(x, y))
    +    val max = data.reduce((x, y) => Math.max(x, y))
    +
    +    val stats = min.mapWithBcVariable(max) {
    +      (minimum, maximum) => (minimum - 2 * (maximum - minimum), maximum + 2 * (maximum - minimum))
    +    }
    +
    +    data.mapPartition(new RichMapPartitionFunction[Double, ContinuousHistogram] {
    --- End diff --
    
    Yes go ahead.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529394
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    --- End diff --
    
    `min` and `max` are always accessible in the scope of the class. So there is no need to use some more memory to save the same values 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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-115180791
  
    The easiest way is probably to check out her branch or the PR and then rebase your work on hers.


---
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-2030][ml]Online Histogram: Discrete and...

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

    https://github.com/apache/flink/pull/861#issuecomment-114900156
  
    OK I've created an issue [here](https://issues.apache.org/jira/browse/FLINK-2274), you can link this PR to that one, and we can close both of them once this merged.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137712869
  
    I also like to take a look before merging.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37529365
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    --- End diff --
    
    Capacity signifies what is the maximum number of bins allowed, while bins signifies what is actually in use.


---
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-2030][FLINK-2274][core][utils]Histogram...

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

    https://github.com/apache/flink/pull/861#issuecomment-137763865
  
    1. @thvasilo , I have changed the scala `DataSetUtils` class to make it a bit more easy to use. Now, it doesn't require an import of `org.apache.flink.api.scala.DataSetUtils.utilsToDataSet`, rather a more intuitive `org.apache.flink.api.scala.DataSetUtils._`.
    Lemme know if this is a good fix to have it along with this PR.
    https://gist.github.com/sachingoel0101/987ebb7cbfacb793d610
    2. I will write the docs for the Utility part.
    
    3. Right now, I have duplicated the implementation from `DataSetUtils.java` directly to `DataSetUtils.scala`. If you find a way to directly access a scala `DataSet[Double]` from java, let me know. :)


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37138093
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,108 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    + <ul>
    +  <li>
    +   <strong>Continuous Histograms</strong>: These histograms are formed on a data set `X: DataSet[Double]` 
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, <i>scaled</i> probability].
    +   <br>
    +    A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +  </li>
    +  <li>
    +    <strong>Categorical Histograms</strong>: These histograms are formed on a data set `X:DataSet[Double]` 
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A categorical histogram can be formed by calling `X.createHistogram(0)`.
    +  </li>
    + </ul>
    +
    +### Data Statistics
    +
    + The `dataStats` function operates on a data set `X: DataSet[Vector]` and returns column-wise
    + statistics for `X`. Every field of `X` is allowed to be defined as either <i>discrete</i> or
    + <i>continuous</i>.
    + <br>
    + Statistics can be evaluated by calling `DataStats.dataStats(X)` or 
    + `DataStats.dataStats(X, discreteFields`). The latter is used when some fields are needed to be 
    + declared discrete-valued, and is provided as an array of indices of fields which are discrete.
    + <br>
    + The following information is available as part of `DataStats`:
    + <ul>
    +    <li>Number of elements in `X`</li>
    +    <li>Dimension of `X`</li>
    +    <li>Column-wise statistics where for discrete fields, we report counts for each category, and
    +     the Gini impurity and Entropy of the field, while for continuous fields, we report the
    +     minimum, maximum, mean and variance.
    +    </li>
    + </ul>
    +
    +## Examples
    +
    +{% highlight scala %}
    +
    +import org.apache.flink.ml.statistics._
    +import org.apache.flink.ml._
    +
    +val X: DataSet[Double] = ...
    +// Create continuous histogram
    +val histogram = X.createHistogram(5)     // creates a histogram with five bins
    +histogram.quantile(0.3)                  // returns the 30th quantile
    +histogram.sum(4)                         // returns number of elements less than 4
    --- End diff --
    
    Is this example valid? `RichDoubleDataSet.createHistogram` returns `DataSet[OnlineHistogram]` and it doesn't have methods such as `quantile`, `sum`, ..., etc..


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37534582
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      return total
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = data.iterator.map(_._2).sum
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    data ++= counts
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    --- End diff --
    
    This will be painfully slow. Each `find` has a complexity of `O(n)`. Consider binary search.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37558899
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    --- End diff --
    
    To be honest, I don't get your explanation.
    
    It doesn't make any sense to write `Math.sqrt(Math.pow(x,2)*y)` because this is the same as `x*Math.sqrt(y)`.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#issuecomment-132193111
  
    They're not coupled at all. But both are related to statistics over data sets. This is why I combined them both in one. If you're wondering, there is a JIRA for the column wise statistics as well. [FLINK-2379].
    Unless it is absolutely necessary, I'd like to keep them both in 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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537041
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    --- End diff --
    
    no need for iterator


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37144175
  
    --- Diff: docs/libs/ml/statistics.md ---
    @@ -0,0 +1,100 @@
    +---
    +mathjax: include
    +htmlTitle: FlinkML - Statistics
    +title: <a href="../ml">FlinkML</a> - Statistics
    +---
    +<!--
    +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
    +
    + The statistics utility provides features such as building histograms over data, determining
    + mean, variance, gini impurity, entropy etc. of data.
    +
    +## Methods
    +
    + The Statistics utility provides two major functions: `createHistogram` and `dataStats`.
    +
    +### Creating a histogram
    +
    + There are two types of histograms:
    +   1. <strong>Continuous Histograms</strong>: These histograms are formed on a data set `X:
    +   DataSet[Double]`
    +   when the values in `X` are from a continuous range. These histograms support
    +   `quantile` and `sum`  operations. Here `quantile(q)` refers to a value $x_q$ such that $|x: x
    +   \leq x_q| = q * |X|$. Further, `sum(s)` refers to the number of elements $x \leq s$, which can
    +    be construed as a cumulative probability value at $s$[Of course, <i>scaled</i> probability].
    +   <br>
    +   2. A continuous histogram can be formed by calling `X.createHistogram(b)` where `b` is the
    +    number of bins.
    +    <strong>Categorical Histograms</strong>: These histograms are formed on a data set `X:DataSet[Double]` 
    +    when the values in `X` are from a discrete distribution. These histograms
    +    support `count(c)` operation which returns the number of elements associated with cateogry `c`.
    +    <br>
    +        A categorical histogram can be formed by calling `X.createHistogram(0)`.
    +
    +### Data Statistics
    +
    + The `dataStats` function operates on a data set `X: DataSet[Vector]` and returns column-wise
    + statistics for `X`. Every field of `X` is allowed to be defined as either <i>discrete</i> or
    + <i>continuous</i>.
    --- End diff --
    
    `<i>` tag


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137762
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  *   Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  *   Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  *   Upper limit on the elements
    +  */
    +case class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += (data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    +
    +  /** Bin counter access function
    +    *
    +    * @param bin bin number to access
    +    * @return `m_bin` = counter of bin
    +    */
    +  private[statistics] def getCounter(bin: Int): Int = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._2
    +  }
    +
    +  /** Number of bins currently being used
    +    *
    +    * @return number of bins
    +    */
    +  override def bins: Int = {
    +    data.size
    +  }
    +
    +  /** Returns the string representation of the histogram.
    +    *
    +    */
    +  override def toString: String = {
    +    s"Size:" + bins + " " + data.toString
    +  }
    +
    +  /** Loads values and counters into the histogram.
    +    * This action can only be performed when there the histogram is empty
    +    *
    +    * @param counts Array of tuple of (value,count)
    +    */
    +  def loadData(counts: Array[(Double, Int)]): Unit = {
    +    if (data.nonEmpty) {
    +      throw new RuntimeException("Data can only be loaded during initialization")
    +    }
    +    counts.iterator.foreach(x => data += ((x._1, x._2)))
    +    require(checkSanity, "Invalid initialization from data")
    +  }
    +
    +  /** Bin value access function
    +    *
    +    * @param bin bin number to access
    +    * @return `v_bin` = value of bin
    +    */
    +  private[statistics] def getValue(bin: Int): Double = {
    +    require(0 <= bin && bin < bins, bin + " not in [0, " + bins + ")")
    +    data.apply(bin)._1
    +  }
    +
    +  /** Returns the lower limit on values of the histogram
    +    *
    +    * @return lower limit on values
    +    */
    +  private def lower: Double = {
    +    min
    +  }
    +
    +  /** Returns the upper limit on values of the histogram
    +    *
    +    * @return upper limit on values
    +    */
    +  private def upper: Double = {
    +    max
    +  }
    +
    +  /** Checks whether the values loaded into data by loadData are properly sorted or not.
    +    * All values should be in (lower,upper)
    +    * All counters loaded into data by loadData should be positive
    +    *
    +    */
    +  private def checkSanity: Boolean = {
    +    val size = bins
    +    if (size == 0) return true
    +    if (lower >= getValue(0)) return false
    +    if (upper <= getValue(size - 1)) return false
    +    for (i <- 0 to size - 2) {
    +      if (getValue(i + 1) < getValue(i)) return false // equality will get merged later on
    +      if (getCounter(i) <= 0) return false
    +    }
    +    if (getCounter(size - 1) <= 0) return false
    +    // now bring the histogram to capacity and merge all <i>very-close</i> elements
    +    while (mergeElements()) {}
    +    true
    +  }
    +
    +  /** Searches for the index with value just greater than p
    +    * If `p >= v_{bins-1}`, return bins
    +    *
    +    * @param p value to search for
    +    * @return the bin with value just greater than p
    +    */
    +  private def find(p: Double): Int = {
    +    val size: Int = bins
    +    if (size == 0) {
    +      return 0
    +    }
    +    if (p < getValue(0)) {
    +      return 0
    +    }
    +    for (i <- 0 to size - 2) {
    +      if (p >= getValue(i) && p < getValue(i + 1)) {
    +        return i + 1
    +      }
    +    }
    +    size
    +  }
    +
    +  /** Merges the closest two elements in the histogram
    +    * If we're over capacity, merge without thought
    +    * Otherwise, only merge when two elements have almost equal values. We're talking doubles, so
    +    * no exact equalities
    +    *
    +    * @return Whether we succeeded in merging any two elements
    +    */
    +  private def mergeElements(): Boolean = {
    +    val size: Int = bins
    +    var minDiffIndex: Int = -1
    +    var minDiff: Double = MaxValue
    +    for (i <- 0 to size - 2) {
    +      val currDiff: Double = getValue(i + 1) - getValue(i)
    +      if (currDiff < minDiff) {
    +        minDiff = currDiff
    +        minDiffIndex = i
    +      }
    +    }
    +    if (bins > capacity || minDiff < 1e-9) {
    +      val weightedValue = getValue(minDiffIndex + 1) * getCounter(minDiffIndex + 1) +
    +        getValue(minDiffIndex) * getCounter(minDiffIndex)
    +      val counterSum = getCounter(minDiffIndex + 1) + getCounter(minDiffIndex)
    +      data.update(minDiffIndex, (weightedValue / counterSum, counterSum))
    +      data.remove(minDiffIndex + 1)
    +      return true
    +    }
    --- End diff --
    
    Saving the return value of `getValue` method and `getCounter` method into immutable variables and reusing them would be better.


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37532900
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  * Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  * Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  * Upper limit on the elements
    +  */
    +class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  private val lower = min
    +  private val upper = max
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += data.apply(i)
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    --- End diff --
    
    Again `IllegalArgumentException`


---
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-2030][FLINK-2274][ml]Online Histograms ...

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

    https://github.com/apache/flink/pull/861#discussion_r37537431
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/DiscreteHistogram.scala ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.statistics
    +
    +import scala.collection.mutable
    +
    +/** Implementation of a discrete valued online histogram
    +  *
    +  * =Parameters=
    +  * -[[numCategories]]:
    +  * Number of categories in the histogram
    +  */
    +class DiscreteHistogram(numCategories: Int) extends OnlineHistogram {
    +
    +  require(numCategories > 0, "Capacity must be greater than zero")
    +  val data = new mutable.HashMap[Double, Int]()
    +
    +  /** Number of categories in the histogram
    +    *
    +    * @return number of categories
    +    */
    +  override def bins: Int = {
    +    numCategories
    +  }
    +
    +  /** Increment count of category c
    +    *
    +    * @param c category whose count needs to be incremented
    +    */
    +  override def add(c: Double): Unit = {
    +    data.get(c) match {
    +      case None =>
    +        require(data.size < numCategories, "Insufficient capacity. Failed to add.")
    +        data.put(c, 1)
    +      case Some(value) =>
    +        data.update(c, value + 1)
    +    }
    +  }
    +
    +  /** Merges the histogram with h and returns a new histogram
    +    *
    +    * @param h histogram to be merged
    +    * @param B number of categories in the resultant histogram.
    +    *          (Default: ```0```, number of categories will be the size of union of categories in
    +    *          both histograms)
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int = 0): DiscreteHistogram = {
    +    h match {
    +      case h1: DiscreteHistogram => {
    +        val finalMap = new mutable.HashMap[Double, Int]()
    +        data.iterator.foreach(x => finalMap.put(x._1, x._2))
    +        h1.data.iterator.foreach(x => {
    +          finalMap.get(x._1) match {
    +            case None => finalMap.put(x._1, x._2)
    +            case Some(value) => finalMap.update(x._1, x._2 + value)
    +          }
    +        })
    +        require(B == 0 || finalMap.size <= B, "Insufficient capacity. Failed to merge")
    +        val finalSize = if (B > 0) B else finalMap.size
    +        val ret = new DiscreteHistogram(finalSize)
    +        ret.loadData(finalMap.toArray)
    --- End diff --
    
    here you basically copy the data again. This is inefficient.


---
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-2030][ml]Data Set Statistics and Histog...

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

    https://github.com/apache/flink/pull/861#discussion_r37137749
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/statistics/ContinuousHistogram.scala ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.statistics
    +
    +import scala.Double.MaxValue
    +import scala.collection.mutable
    +
    +/** Implementation of a continuous valued online histogram
    +  * Adapted from Ben-Haim and Yom-Tov's Streaming Decision Tree Algorithm
    +  * Refer http://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
    +  *
    +  * =Parameters=
    +  * -[[capacity]]:
    +  *   Number of bins to be used in the histogram
    +  *
    +  * -[[min]]:
    +  *   Lower limit on the elements
    +  *
    +  * -[[max]]:
    +  *   Upper limit on the elements
    +  */
    +case class ContinuousHistogram(capacity: Int, min: Double, max: Double) extends OnlineHistogram {
    +
    +  require(capacity > 0, "Capacity should be a positive integer")
    +  require(lower < upper, "Lower must be less than upper")
    +
    +  val data = new mutable.ArrayBuffer[(Double, Int)]()
    +
    +  /** Adds a new item to the histogram
    +    *
    +    * @param p value to be added
    +    */
    +  override def add(p: Double): Unit = {
    +    require(p > lower && p < upper, p + " not in (" + lower + "," + upper + ")")
    +    // search for the index where the value is just higher than p
    +    val search = find(p)
    +    // add the new value there, shifting everything to the right
    +    data.insert(search, (p, 1))
    +    // If we're over capacity or any two elements are within 1e-9 of each other, merge.
    +    // This will take care of the case if p was actually equal to some value in the histogram and
    +    // just increment the value there
    +    mergeElements()
    +  }
    +
    +  /** Merges the histogram with h and returns a histogram with capacity B
    +    *
    +    * @param h histogram to be merged
    +    * @param B capacity of the resultant histogram
    +    * @return Merged histogram with capacity B
    +    */
    +  override def merge(h: OnlineHistogram, B: Int): ContinuousHistogram = {
    +    h match {
    +      case temp: ContinuousHistogram => {
    +        val m: Int = bins
    +        val n: Int = temp.bins
    +        var i, j: Int = 0
    +        val mergeList = new mutable.ArrayBuffer[(Double, Int)]()
    +        while (i < m || j < n) {
    +          if (i >= m) {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          } else if (j >= n || getValue(i) <= temp.getValue(j)) {
    +            mergeList += (data.apply(i))
    +            i = i + 1
    +          } else {
    +            mergeList += ((temp.getValue(j), temp.getCounter(j)))
    +            j = j + 1
    +          }
    +        }
    +        // the size will be brought to capacity while constructing the new histogram itself
    +        val finalLower = Math.min(lower, temp.lower)
    +        val finalUpper = Math.max(upper, temp.upper)
    +        val ret = new ContinuousHistogram(B, finalLower, finalUpper)
    +        ret.loadData(mergeList.toArray)
    +        ret
    +      }
    +      case default =>
    +        throw new RuntimeException("Only a continuous histogram is allowed to be merged with a " +
    +          "continuous histogram")
    +
    +    }
    +  }
    +
    +  /** Returns the qth quantile of the histogram
    +    *
    +    * @param q Quantile value in (0,1)
    +    * @return Value at quantile q
    +    */
    +  def quantile(q: Double): Double = {
    +    require(bins > 0, "Histogram is empty")
    +    require(q > 0 && q < 1, "Quantile must be between 0 and 1")
    +    var total = 0
    +    for (i <- 0 to bins - 1) {
    +      total = total + getCounter(i)
    +    }
    +    val wantedSum = (q * total).round.toInt
    +    var currSum = count(getValue(0))
    +
    +    if (wantedSum < currSum) {
    +      require(lower > -MaxValue, "Set a lower bound before proceeding")
    +      return Math.sqrt(2 * wantedSum * Math.pow(getValue(0) - lower, 2) / getCounter(0)) + lower
    +    }
    +
    +    /** Walk the histogram to find sums at every bin value
    +      * As soon as you hit the interval where you should be
    +      * Walk along the trapezoidal line
    +      * This leads to solving a quadratic equation.
    +      */
    +    for (i <- 1 to bins - 1) {
    +      val tmpSum = count(getValue(i))
    +      if (currSum <= wantedSum && wantedSum < tmpSum) {
    +        val neededSum = wantedSum - currSum
    +        val a: Double = getCounter(i) - getCounter(i - 1)
    +        val b: Double = 2 * getCounter(i - 1)
    +        val c: Double = -2 * neededSum
    +        if (a == 0) {
    +          return getValue(i - 1) + (getValue(i) - getValue(i - 1)) * (-c / b)
    +        } else {
    +          return getValue(i - 1) +
    +            (getValue(i) - getValue(i - 1)) * (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a)
    +        }
    +      } else {
    +        currSum = tmpSum
    +      }
    +    }
    +    require(upper < MaxValue, "Set an upper bound before proceeding")
    +    // this means wantedSum > sum(getValue(bins-1))
    +    // this will likely fail to return a bounded value.
    +    // Make sure you set some proper limits on min and max.
    +    getValue(bins - 1) + Math.sqrt(
    +      Math.pow(upper - getValue(bins - 1), 2) * 2 * (wantedSum - currSum) / getCounter(bins - 1))
    +  }
    +
    +  /** Returns the probability (and by extension, the number of points) for the value b
    +    * Since this is a continuous histogram, return the cumulative probability at b
    +    *
    +    * @return Cumulative number of points at b
    +    */
    +  def count(b: Double): Int = {
    +    require(bins > 0, "Histogram is empty")
    +    if (b < lower) {
    +      return 0
    +    }
    +    if (b >= upper) {
    +      var ret = 0
    +      for (i <- 0 to bins - 1) {
    +        ret = ret + getCounter(i)
    +      }
    +      return ret
    +    }
    +    /** Suppose x is the index with value just less than or equal to b
    +      * Then, sum everything up to x-1
    +      * Add half the value at x
    +      * Find area of the trapezoid for x and the value b
    +      */
    +
    +    val index = find(b) - 1
    +    var m_b, s: Double = 0
    +    if (index == -1) {
    +      m_b = getCounter(index + 1) * (b - lower) / (getValue(index + 1) - lower)
    +      s = m_b * (b - lower) / (2 * (getValue(index + 1) - lower))
    +      return s.round.toInt
    +    } else if (index == bins - 1) {
    +      m_b = getCounter(index) +
    +        (-getCounter(index)) * (b - getValue(index)) / (upper - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (upper - getValue(index)))
    +    } else {
    +      m_b = getCounter(index) + (getCounter(index + 1) - getCounter(index)) *
    +        (b - getValue(index)) / (getValue(index + 1) - getValue(index))
    +      s = (getCounter(index) + m_b) *
    +        (b - getValue(index)) / (2 * (getValue(index + 1) - getValue(index)))
    +    }
    +    for (i <- 0 to index - 1) {
    +      s = s + getCounter(i)
    +    }
    +    s = s + getCounter(index) / 2
    +    s.round.toInt
    +  }
    +
    +  /** Returns the total number of entries in the histogram
    +    *
    +    * @return Total number of points added in the histogram
    +    */
    +  override def total: Int = {
    +    var result = 0
    +    for (i <- 0 to bins - 1) {
    +      result = result + getCounter(i)
    +    }
    +    result
    +  }
    --- End diff --
    
    `total` method can be replaced by following. `getCounter` method checks range. But in this case, we don't need to check range.
    
    ```scala
    override def total: Int = data.iterator.map(_._2).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.
---