You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by sethah <gi...@git.apache.org> on 2015/10/06 23:15:57 UTC

[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

GitHub user sethah opened a pull request:

    https://github.com/apache/spark/pull/9003

    [SPARK-10641][WIP][SQL] Add Skewness and Kurtosis Support

    Implementing skewness and kurtosis support based on following algorithm:
    https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics


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

    $ git pull https://github.com/sethah/spark SPARK-10641

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

    https://github.com/apache/spark/pull/9003.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 #9003
    
----
commit 692a3953f38ec0fc7579510ed696e1d4787cacce
Author: sethah <se...@gmail.com>
Date:   2015-09-16T23:52:43Z

    Added skewness and kurtosis aggregate functions

commit 83ef359fdb568e3b8dc809b07e0a80899ee984c5
Author: sethah <se...@gmail.com>
Date:   2015-09-18T20:05:10Z

    Adding kahan updates to higher order aggregate stats

commit 5efee6de1dba253926eb2399fac788e86cbe1027
Author: sethah <se...@gmail.com>
Date:   2015-09-21T16:48:14Z

    adding zero division protection

commit c98fa11815a20484707282490f9e8fdfd3df96a0
Author: sethah <se...@gmail.com>
Date:   2015-10-06T04:12:48Z

    Adding order check to reduce calculation overhead

commit 60fd7548ce896241a462621e026b9577ab14eabf
Author: sethah <se...@gmail.com>
Date:   2015-10-06T04:20:41Z

    style and scaladoc fixes

commit e6c8d277d01ecffab7473bdd47de2e5069692ac6
Author: sethah <se...@gmail.com>
Date:   2015-10-06T20:56:38Z

    updating kurtosis test

----


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42889941
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    +      buffer.getDouble(mutableAggBufferOffset + momentIdx)
    +    }
    +    getStatistic(n, moments)
    +  }
    +}
    +
    +abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) {
    +
    +  override protected val momentOrder = 2
    +
    +  protected def isBiased: Boolean
    +
    +  protected def isStd: Boolean
    +
    +  override protected val sufficientMoments = Array(2)
    +
    +  override def getStatistic(n: Double, moments: Array[Double]): Double = {
    +    require(moments.length == sufficientMoments.length,
    +      s"$prettyName requires one central moment, received: ${moments.length}")
    +
    +    val m2 = moments.head
    +    val divisor = if (isBiased) n else n - 1
    +    val variance = if (n == 0.0) {
    +      Double.NaN
    +    } else {
    +      m2 / divisor
    +    }
    +
    +    if (isStd) {
    +      math.sqrt(variance)
    +    } else {
    +      variance
    +    }
    +  }
    +}
    +
    +case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,
    +                    inputAggBufferOffset: Int = 0) extends SecondMoment(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance"
    +
    +  override protected val isBiased = false
    +
    +  override protected val isStd = false
    +}
    +
    +case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,
    +                        inputAggBufferOffset: Int = 0) extends SecondMoment(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override protected val isBiased = false
    +
    +  override protected val isStd = false
    +}
    +
    +case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,
    +                       inputAggBufferOffset: Int = 0) extends SecondMoment(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override protected val isBiased = true
    +
    +  override protected val isStd = false
    +}
    +
    +case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,
    +                    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "skewness"
    +
    +  override protected val momentOrder = 3
    +
    +  override protected val sufficientMoments = Array(2, 3)
    +
    +  override def getStatistic(n: Double, moments: Array[Double]): Double = {
    +    require(moments.length == sufficientMoments.length,
    +      s"$prettyName requires two central moments, received: ${moments.length}")
    +    val Array(m2, m3) = moments
    +    if (n == 0.0 || m2 == 0.0) {
    +      Double.NaN
    +    } else {
    +      math.sqrt(n) * m3 / math.sqrt(m2 * m2 * m2)
    +    }
    +  }
    +}
    +
    +case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,
    --- End diff --
    
    Done.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149786976
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r41324803
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ---
    @@ -221,4 +221,40 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
           emptyTableData.agg(sumDistinct('a)),
           Row(null))
       }
    +
    +  test("moments") {
    +    checkAnswer(
    +      testData2.agg(skewness('a)),
    +      Row(0.0))
    +
    +    checkAnswer(
    --- End diff --
    
    This test needs a tolerance added in order for it to pass. Appreciate any tips on how to do 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149788654
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#discussion_r43413026
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala ---
    @@ -991,3 +991,73 @@ case class StddevFunction(
         }
       }
     }
    +
    +// placeholder
    +case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "kurtosis"
    +
    +  override def toString: String = s"KURTOSIS($child)"
    +}
    +
    +// placeholder
    +case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "skewness"
    +
    +  override def toString: String = s"SKEWNESS($child)"
    +}
    +
    +// placeholder
    +case class Variance(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance"
    +
    +  override def toString: String = s"VARIANCE($child)"
    +}
    +
    +// placeholder
    +case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override def toString: String = s"VAR_POP($child)"
    +}
    +
    +// placeholder
    +case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override def toString: String = s"VAR_SAMP($child)"
    +}
    --- End diff --
    
    @yhuai Thanks for the suggestion! I think that's working now if you want to review 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42453557
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    --- End diff --
    
    Do you think we should link to page in the code as well as in the API doc?


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150736289
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149424463
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152290694
  
    Merge into master. Thanks! @JihongMA @sethah Could you send a PR to replace `Stddev` implementation using `CentralMomentAgg`? I ran the following and the difference is significant:
    
    ~~~scala
    val df = sqlContext.range(100000000)
    df.select(var_samp("id")).show();
    df.select(stddev_samp("id")).show()
    ~~~


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42889596
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    --- End diff --
    
    done.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#discussion_r43402646
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala ---
    @@ -991,3 +991,73 @@ case class StddevFunction(
         }
       }
     }
    +
    +// placeholder
    +case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "kurtosis"
    +
    +  override def toString: String = s"KURTOSIS($child)"
    +}
    +
    +// placeholder
    +case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "skewness"
    +
    +  override def toString: String = s"SKEWNESS($child)"
    +}
    +
    +// placeholder
    +case class Variance(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance"
    +
    +  override def toString: String = s"VARIANCE($child)"
    +}
    +
    +// placeholder
    +case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override def toString: String = s"VAR_POP($child)"
    +}
    +
    +// placeholder
    +case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override def toString: String = s"VAR_SAMP($child)"
    +}
    --- End diff --
    
    Can you change `with AggregateExpression` to `with AggregateExpression1`? Then, in the `newInstance` method, we throw an `UnsupportedOperationException` to let users know that they need to set `spark.sql.useAggregate2` to `true`.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149779486
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433431
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    +      secondMoment1 + secondMoment2 + delta * deltaN * zeroMoment1 * zeroMoment2
    +    } else {
    +      0.0
    +    }
    +
    +    val thirdMoment = if (maxMoment >= 3) {
    +      thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * zeroMoment1 * zeroMoment2 *
    +        (zeroMoment1 - zeroMoment2) + 3.0 * deltaN *
    +        (zeroMoment1 * secondMoment2 - zeroMoment2 * secondMoment1)
    +    } else {
    +      0.0
    +    }
    +
    +    val fourthMoment = if (maxMoment >= 4) {
    +      fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * zeroMoment1 *
    +        zeroMoment2 * (zeroMoment1 * zeroMoment1 - zeroMoment1 * zeroMoment2 +
    +          zeroMoment2 * zeroMoment2) + deltaN * deltaN * 6.0 *
    +        (zeroMoment1 * zeroMoment1 * secondMoment2 + zeroMoment2 * zeroMoment2 * secondMoment1) +
    +        4.0 * deltaN * (zeroMoment1 * thirdMoment2 - zeroMoment2 * thirdMoment1)
    +    } else {
    +      0.0
    +    }
    +
    +    buffer1.setDouble(mutableAggBufferOffset, zeroMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, firstMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +  }
    +}
    +
    +case class Stddev(child: Expression) extends CentralMomentAgg(child) {
    +
    +  override def prettyName: String = "stddev"
    +
    +  protected val maxMoment = 2
    +
    +  def eval(buffer: InternalRow): Any = {
    +    // stddev = sqrt(M2 / (M0 - 1))
    +    val M0 = buffer.getDouble(mutableAggBufferOffset)
    +    val M2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +
    +    if (M0 == 0.0) {
    +      0.0
    --- End diff --
    
    Should be `NaN` instead of `0.0` when `n = 0` or `n = 1`. Check R's output.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150662564
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44234/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42453520
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    +      secondMoment1 + secondMoment2 + delta * deltaN * zeroMoment1 * zeroMoment2
    +    } else {
    +      0.0
    +    }
    +
    +    val thirdMoment = if (maxMoment >= 3) {
    +      thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * zeroMoment1 * zeroMoment2 *
    +        (zeroMoment1 - zeroMoment2) + 3.0 * deltaN *
    +        (zeroMoment1 * secondMoment2 - zeroMoment2 * secondMoment1)
    +    } else {
    +      0.0
    +    }
    +
    +    val fourthMoment = if (maxMoment >= 4) {
    +      fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * zeroMoment1 *
    +        zeroMoment2 * (zeroMoment1 * zeroMoment1 - zeroMoment1 * zeroMoment2 +
    +          zeroMoment2 * zeroMoment2) + deltaN * deltaN * 6.0 *
    +        (zeroMoment1 * zeroMoment1 * secondMoment2 + zeroMoment2 * zeroMoment2 * secondMoment1) +
    +        4.0 * deltaN * (zeroMoment1 * thirdMoment2 - zeroMoment2 * thirdMoment1)
    +    } else {
    +      0.0
    +    }
    +
    +    buffer1.setDouble(mutableAggBufferOffset, zeroMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, firstMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +  }
    +}
    +
    +case class Stddev(child: Expression) extends CentralMomentAgg(child) {
    +
    +  override def prettyName: String = "stddev"
    +
    +  protected val maxMoment = 2
    +
    +  def eval(buffer: InternalRow): Any = {
    +    // stddev = sqrt(M2 / (M0 - 1))
    +    val M0 = buffer.getDouble(mutableAggBufferOffset)
    +    val M2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +
    +    if (M0 == 0.0) {
    +      0.0
    --- End diff --
    
    R and SciPy differ on this. R outputs NaN in divide by zero situations where SciPy outputs 0. I will assume we stick with R implementation.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150729081
  
    **[Test build #44267 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44267/consoleFull)** for PR 9003 at commit [`cf8a14b`](https://github.com/apache/spark/commit/cf8a14bb24924079af2c30234a083a1e6c6d4c23).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `case class Variance(child: Expression,`\n  * `case class VarianceSamp(child: Expression,`\n  * `case class VariancePop(child: Expression,`\n  * `case class Skewness(child: Expression,`\n  * `case class Kurtosis(child: Expression,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149788655
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44045/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150324361
  
    **[Test build #44164 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44164/consoleFull)** for PR 9003 at commit [`2c9046f`](https://github.com/apache/spark/commit/2c9046ff952d80ff6c9382fc9e1ba31dc8ade7b8).
     * This patch **fails Scala style tests**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VariancePop(child: Expression
 ) extends UnaryExpression with AggregateExpression `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42889712
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    --- End diff --
    
    Looking at the code, `Cast.eval` should return the correct type or null, so this extra case statements has been removed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42897839
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,330 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * Returns `Double.NaN` when N = 0 or N = 1
    + *  -third and fourth moments return `Double.NaN` when second moment is zero
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * Size of aggregation buffer.
    +   */
    +  private[this] val bufferSize = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(bufferSize) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  // buffer offsets
    +  private[this] val nOffset = mutableAggBufferOffset
    +  private[this] val meanOffset = mutableAggBufferOffset + 1
    +  private[this] val secondMomentOffset = mutableAggBufferOffset + 2
    +  private[this] val thirdMomentOffset = mutableAggBufferOffset + 3
    +  private[this] val fourthMomentOffset = mutableAggBufferOffset + 4
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +  private[this] var n = 0.0
    +  private[this] var mean = 0.0
    +  private[this] var m2 = 0.0
    +  private[this] var m3 = 0.0
    +  private[this] var m4 = 0.0
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until bufferSize) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +      }
    +      n = buffer.getDouble(nOffset)
    +      mean = buffer.getDouble(meanOffset)
    +
    +      n += 1.0
    +      buffer.setDouble(nOffset, n)
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(meanOffset, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(secondMomentOffset)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(secondMomentOffset, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(thirdMomentOffset)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(thirdMomentOffset, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(fourthMomentOffset)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(fourthMomentOffset, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(nOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(meanOffset)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +
    +    n = n1 + n2
    +    buffer1.setDouble(nOffset, n)
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    mean = mean1 + deltaN * n
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(secondMomentOffset)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      m2 = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(secondMomentOffset, m2)
    +    }
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(thirdMomentOffset)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      m3 = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(thirdMomentOffset, m3)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(fourthMomentOffset)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      m4 = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(fourthMomentOffset, m4)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   * @param centralMoments Length `momentOrder + 1` array of central moments needed to
    +   *                       compute the aggregate stat.
    --- End diff --
    
    It is useful to mention that whether they are divided by `n` or not.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42584878
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    +      buffer.getDouble(mutableAggBufferOffset + momentIdx)
    +    }
    +    getStatistic(n, moments)
    +  }
    +}
    +
    +abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) {
    --- End diff --
    
    All `StdDev` and `Variance` subclasses can extend `SecondMoment` abstract class since their `getStatistic` methods are nearly identical.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42813128
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    +      buffer.getDouble(mutableAggBufferOffset + momentIdx)
    +    }
    +    getStatistic(n, moments)
    +  }
    +}
    +
    +abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) {
    +
    +  override protected val momentOrder = 2
    +
    +  protected def isBiased: Boolean
    +
    +  protected def isStd: Boolean
    +
    +  override protected val sufficientMoments = Array(2)
    +
    +  override def getStatistic(n: Double, moments: Array[Double]): Double = {
    +    require(moments.length == sufficientMoments.length,
    +      s"$prettyName requires one central moment, received: ${moments.length}")
    +
    +    val m2 = moments.head
    +    val divisor = if (isBiased) n else n - 1
    +    val variance = if (n == 0.0) {
    +      Double.NaN
    +    } else {
    +      m2 / divisor
    +    }
    +
    +    if (isStd) {
    +      math.sqrt(variance)
    +    } else {
    +      variance
    +    }
    +  }
    +}
    +
    +case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,
    +                    inputAggBufferOffset: Int = 0) extends SecondMoment(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance"
    +
    +  override protected val isBiased = false
    +
    +  override protected val isStd = false
    +}
    +
    +case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,
    +                        inputAggBufferOffset: Int = 0) extends SecondMoment(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override protected val isBiased = false
    +
    +  override protected val isStd = false
    +}
    +
    +case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,
    +                       inputAggBufferOffset: Int = 0) extends SecondMoment(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override protected val isBiased = true
    +
    +  override protected val isStd = false
    +}
    +
    +case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,
    +                    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "skewness"
    +
    +  override protected val momentOrder = 3
    +
    +  override protected val sufficientMoments = Array(2, 3)
    +
    +  override def getStatistic(n: Double, moments: Array[Double]): Double = {
    +    require(moments.length == sufficientMoments.length,
    +      s"$prettyName requires two central moments, received: ${moments.length}")
    +    val Array(m2, m3) = moments
    +    if (n == 0.0 || m2 == 0.0) {
    +      Double.NaN
    +    } else {
    +      math.sqrt(n) * m3 / math.sqrt(m2 * m2 * m2)
    +    }
    +  }
    +}
    +
    +case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,
    --- End diff --
    
    * Need to document which Kurtosis we implemented.
    * Chop down args to match Spark code style.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150636243
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151248172
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42717893
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ---
    @@ -221,4 +221,40 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
           emptyTableData.agg(sumDistinct('a)),
           Row(null))
       }
    +
    +  test("moments") {
    +    checkAnswer(
    +      testData2.agg(skewness('a)),
    +      Row(0.0))
    +
    +    checkAnswer(
    --- End diff --
    
    I think sharing `checkAnswerWithTol` as a helper function in `QueryTest` sounds like a good plan, I'm not super sure on the name (since checkAnswer works with an expected Row and the signature is a bit further off than just adding a tolerance param), but it seems probably ok with the current name.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152287002
  
    Merged build finished. Test PASSed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42665351
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    --- End diff --
    
    for (aggIndex <- 0 until numMoments) {
      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150105958
  
    Build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152209172
  
    **[Test build #44595 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44595/consoleFull)** for PR 9003 at commit [`ff363cc`](https://github.com/apache/spark/commit/ff363cca57e2b1c2bb28e281d014d33b930fd603).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `case class Variance(child: Expression,`\n  * `case class VarianceSamp(child: Expression,`\n  * `case class VariancePop(child: Expression,`\n  * `case class Skewness(child: Expression,`\n  * `case class Kurtosis(child: Expression,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150324365
  
    Build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149783441
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152175141
  
    **[Test build #44595 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44595/consoleFull)** for PR 9003 at commit [`ff363cc`](https://github.com/apache/spark/commit/ff363cca57e2b1c2bb28e281d014d33b930fd603).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149997989
  
    Seth, no need to implement the old interface, simply put a placeholder for resolving it is sufficient, which will go away when this code path is removed. 
    
    here is an example
    
    case class Range(child: Expression) extends UnaryExpression with AggregateExpression {
    +  override def nullable: Boolean = false
    +  override def dataType: DoubleType.type = DoubleType
    +  override def foldable: Boolean = false
    +  override def toString: String = s"RANGE($child)"
    +}


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150729097
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149424324
  
    @mengxr I think having `VarianceSamp` inherit from `Variance` will be a fine solution. I'm not clear on why it is better not to touch `InternalRow` in the subclasses, but if we must avoid it, one solution could be
    
    ```scala
    def eval(buffer: InternalRow): Any = this match {
          case _: VariancePop => { }
          case _: Variance => { }
          case _: Skewness => { }
          case _: Kurtosis => { }
    ```


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150085176
  
     Build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149790428
  
    **[Test build #44046 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44046/consoleFull)** for PR 9003 at commit [`62c515c`](https://github.com/apache/spark/commit/62c515cad1e2fb712256601f139d098fa4fb3749).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `abstract class CentralMomentAgg1(child: Expression)`\n  * `case class Kurtosis(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class Skewness(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class Variance(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class VariancePop(child:
  Expression) extends CentralMomentAgg1(child) `\n  * `case class VarianceSamp(child: Expression) extends CentralMomentAgg1(child) `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151362354
  
    Had an offline discussion with @yhuai. We can remove `fetch_aggregate` from whitelist. We already tested all the functions there in SQL queries. We can create a new JIRA to add it back and handle small numerical differences, but it should be orthogonal to this PR.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42562714
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    --- End diff --
    
    I think it is useful to put a link inline because that wikipedia page is 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150105932
  
    **[Test build #44119 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44119/consoleFull)** for PR 9003 at commit [`1594aba`](https://github.com/apache/spark/commit/1594abafec18dd0ba1c67c7a68c8ff017f88660a).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `abstract class CentralMomentAgg1(child: Expression)`\n  * `case class Kurtosis(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class Skewness(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class Variance(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class VariancePop(child:
  Expression) extends CentralMomentAgg1(child) `\n  * `case class VarianceSamp(child: Expression) extends CentralMomentAgg1(child) `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r41428901
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ---
    @@ -221,4 +221,40 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
           emptyTableData.agg(sumDistinct('a)),
           Row(null))
       }
    +
    +  test("moments") {
    +    checkAnswer(
    +      testData2.agg(skewness('a)),
    +      Row(0.0))
    +
    +    checkAnswer(
    --- End diff --
    
    @sethah currently checkAnswer only supports exact match, looking at the code we can expand it to accept an error bound in QueryTest.scala. 



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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150367576
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150368373
  
    **[Test build #44179 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44179/consoleFull)** for PR 9003 at commit [`3ef2faa`](https://github.com/apache/spark/commit/3ef2faaa83104273ab185a3190f2d133a04693bb).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42812969
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    --- End diff --
    
    Those vars could be `private[this]` member vars.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150085187
  
    Build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150389712
  
    **[Test build #44179 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44179/consoleFull)** for PR 9003 at commit [`3ef2faa`](https://github.com/apache/spark/commit/3ef2faaa83104273ab185a3190f2d133a04693bb).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VariancePop(child: Expression
 ) extends UnaryExpression with AggregateExpression `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151248009
  
    **[Test build #44366 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44366/consoleFull)** for PR 9003 at commit [`3045e3b`](https://github.com/apache/spark/commit/3045e3b1d82ac73d154dc4d2165e920c74bdc118).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `case class Variance(child: Expression,`\n  * `case class VarianceSamp(child: Expression,`\n  * `case class VariancePop(child: Expression,`\n  * `case class Skewness(child: Expression,`\n  * `case class Kurtosis(child: Expression,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433410
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    --- End diff --
    
    not needed if `maxMoment < 3`.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152248103
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-146005754
  
    A few notes:
    * I wrote this implementation before stddev was merged, and hence I ended up with a slightly different implementation. The differences are mostly syntactical (the algorithms for computing second moment are the same), except for how the subclasses are implemented. Once we figure out the best way to implement the subclasses, I will merge the implementations to remove the overlap
    * (https://issues.apache.org/jira/browse/SPARK-10953)[SPARK-10953] Calls for benchmarking codegen vs scala implementations of univariate stats. The results of that may affect this PR.
    * I did not implement the skewness and kurtosis functions using the old way with AggregateFunction1, since I think that will be removed. I can implement this separately 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149424443
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149783421
  
    **[Test build #44039 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44039/consoleFull)** for PR 9003 at commit [`4ae234e`](https://github.com/apache/spark/commit/4ae234e3c8385e3149937821f5cdcee8fb85c053).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42923553
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,330 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * Returns `Double.NaN` when N = 0 or N = 1
    + *  -third and fourth moments return `Double.NaN` when second moment is zero
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * Size of aggregation buffer.
    +   */
    +  private[this] val bufferSize = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(bufferSize) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  // buffer offsets
    +  private[this] val nOffset = mutableAggBufferOffset
    +  private[this] val meanOffset = mutableAggBufferOffset + 1
    +  private[this] val secondMomentOffset = mutableAggBufferOffset + 2
    +  private[this] val thirdMomentOffset = mutableAggBufferOffset + 3
    +  private[this] val fourthMomentOffset = mutableAggBufferOffset + 4
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +  private[this] var n = 0.0
    +  private[this] var mean = 0.0
    +  private[this] var m2 = 0.0
    +  private[this] var m3 = 0.0
    +  private[this] var m4 = 0.0
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until bufferSize) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +      }
    +      n = buffer.getDouble(nOffset)
    +      mean = buffer.getDouble(meanOffset)
    +
    +      n += 1.0
    +      buffer.setDouble(nOffset, n)
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(meanOffset, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(secondMomentOffset)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(secondMomentOffset, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(thirdMomentOffset)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(thirdMomentOffset, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(fourthMomentOffset)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(fourthMomentOffset, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(nOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(meanOffset)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +
    +    n = n1 + n2
    +    buffer1.setDouble(nOffset, n)
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    mean = mean1 + deltaN * n
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(secondMomentOffset)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      m2 = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(secondMomentOffset, m2)
    +    }
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(thirdMomentOffset)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      m3 = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(thirdMomentOffset, m3)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(fourthMomentOffset)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      m4 = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(fourthMomentOffset, m4)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   * @param centralMoments Length `momentOrder + 1` array of central moments needed to
    +   *                       compute the aggregate stat.
    +   */
    +  def getStatistic(n: Double, mean: Double, centralMoments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(nOffset)
    +    val mean = buffer.getDouble(meanOffset)
    +    val moments = Array.ofDim[Double](momentOrder + 1)
    +    moments(0) = n
    +    moments(1) = mean
    +    if (momentOrder >= 2) {
    +      moments(2) = buffer.getDouble(secondMomentOffset)
    +    }
    +    if (momentOrder >= 3) {
    +      moments(3) = buffer.getDouble(thirdMomentOffset)
    +    }
    +    if (momentOrder >= 4) {
    +      moments(4) = buffer.getDouble(fourthMomentOffset)
    +    }
    +
    +    getStatistic(n, mean, moments)
    +  }
    +}
    +
    +case class Variance(child: Expression,
    +    mutableAggBufferOffset: Int = 0,
    +    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance"
    +
    +  override protected val momentOrder = 2
    +
    +  override def getStatistic(n: Double, mean: Double, moments: Array[Double]): Double = {
    +    require(moments.length == momentOrder + 1,
    +      s"$prettyName requires ${momentOrder + 1} central moments, received: ${moments.length}")
    +
    +    if (n == 0.0 || n == 1.0) Double.NaN else moments(2) / (n - 1.0)
    +  }
    +}
    +
    +case class VarianceSamp(child: Expression,
    +    mutableAggBufferOffset: Int = 0,
    +    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override protected val momentOrder = 2
    +
    +  override def getStatistic(n: Double, mean: Double, moments: Array[Double]): Double = {
    +    require(moments.length == momentOrder + 1,
    +      s"$prettyName requires ${momentOrder + 1} central moment, received: ${moments.length}")
    +
    +    if (n == 0.0 || n == 1.0) Double.NaN else moments(2) / (n - 1.0)
    +  }
    +}
    +
    +case class VariancePop(child: Expression,
    +    mutableAggBufferOffset: Int = 0,
    +    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override protected val momentOrder = 2
    +
    +  override def getStatistic(n: Double, mean: Double, moments: Array[Double]): Double = {
    +    require(moments.length == momentOrder + 1,
    +      s"$prettyName requires ${momentOrder + 1} central moment, received: ${moments.length}")
    +
    +    if (n == 0.0 || n == 1.0) Double.NaN else moments(2) / n
    --- End diff --
    
    Updated


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#discussion_r43388459
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala ---
    @@ -991,3 +991,73 @@ case class StddevFunction(
         }
       }
     }
    +
    +// placeholder
    +case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "kurtosis"
    +
    +  override def toString: String = s"KURTOSIS($child)"
    +}
    +
    +// placeholder
    +case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "skewness"
    +
    +  override def toString: String = s"SKEWNESS($child)"
    +}
    +
    +// placeholder
    +case class Variance(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance"
    +
    +  override def toString: String = s"VARIANCE($child)"
    +}
    +
    +// placeholder
    +case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override def toString: String = s"VAR_POP($child)"
    +}
    +
    +// placeholder
    +case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override def toString: String = s"VAR_SAMP($child)"
    +}
    --- End diff --
    
    @yhuai This is the error I get when calling one of the new `skewness` on a dataframe when `spark.sql.useAggregate2=false`: 
    
    ```
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 2, localhost): org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree:{colName}#2
    ```
    
    I'm not exactly sure where we ought to throw the error. I'd appreciate any tips on how to do 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150729098
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44267/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42813023
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    --- End diff --
    
    minor: move this line right after `n += 1.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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150367608
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42812911
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    --- End diff --
    
    This is not necessary because the k-th order moment update depends on all lower order moments, even we don't use it in the final formulation. Since `eval` is only called once, simplicity and readability is more important than performance. So having an `Array` with all central moments is better 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42813076
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    +      buffer.getDouble(mutableAggBufferOffset + momentIdx)
    +    }
    +    getStatistic(n, moments)
    +  }
    +}
    +
    +abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) {
    --- End diff --
    
    This class actually complicates the implementation. Most of the subclasses only few lines of code:
    
    ~~~scala
    case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) {
      override def momentOrder = 2
      override def getStatistics(n: Double, mean: Double, centralMoments: Array[Double]): Double = {
        if (n == 0 || n == 1) Double.NaN else centralMoments(2) / (n - 1.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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42584417
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    --- End diff --
    
    Changed variable names and declared as vars.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42706392
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ---
    @@ -221,4 +221,40 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
           emptyTableData.agg(sumDistinct('a)),
           Row(null))
       }
    +
    +  test("moments") {
    +    checkAnswer(
    +      testData2.agg(skewness('a)),
    +      Row(0.0))
    +
    +    checkAnswer(
    --- End diff --
    
    @holdenk @JihongMA Thanks for the suggestions! I updated `SQLQuerySuite` with a private `checkAnswerWithTol` helper function that solves the problem for the specific tests written so far. However, there are tests in `DataFrameAggregateSuite` that could fail if the test data ever changes - they too could benefit from a `checkAnswer` function that accepts a tolerance. I'm not sure if we can leave as is for now or perhaps the helper function should be added to `QueryTest` object so that it can be used in both test suites. Open to suggestions...


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150939324
  
    @sethah It seems that `VARIANCE` in HIVE returns the population variance. It is the same in MySQL: https://dev.mysql.com/doc/refman/5.0/en/group-by-functions.html#function_variance. So let's keep the same behavior. Then it is a little awkward that `stddev` is sample variance in our implementation. HIVE doesn't have `stddev`, while MySQL uses population stddev for `stddev`. I think it is more important to be compatible with existing SQL engines. We can address the `stddev` in a follow-up PR.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42889772
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    --- End diff --
    
    Added as `private[this]` vars


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433404
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    --- End diff --
    
    Those computation are not needed if `maxMoment < 3`. So we should move then into the `if (maxMoment >= 3) {` block. Again, changing them to private vars helps 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149787044
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42393149
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,304 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  def cloneBufferAttributes: Seq[Attribute] = aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = child.eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: java.lang.Number => d.doubleValue()
    +        case _ => 0.0
    +      }
    --- End diff --
    
    please double check if it will hanle all numeric types correctly, e.g: Decimal?


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42584440
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    --- End diff --
    
    moved unnecessary gets and sets into if 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150655247
  
    LGTM except minor inline comments. Ping @yhuai for a final pass.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150736290
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44272/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149425775
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r43162207
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala ---
    @@ -991,3 +991,73 @@ case class StddevFunction(
         }
       }
     }
    +
    +// placeholder
    +case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "kurtosis"
    +
    +  override def toString: String = s"KURTOSIS($child)"
    +}
    +
    +// placeholder
    +case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "skewness"
    +
    +  override def toString: String = s"SKEWNESS($child)"
    +}
    +
    +// placeholder
    +case class Variance(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance"
    +
    +  override def toString: String = s"VARIANCE($child)"
    +}
    +
    +// placeholder
    +case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override def toString: String = s"VAR_POP($child)"
    +}
    +
    +// placeholder
    +case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression {
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DoubleType.type = DoubleType
    +
    +  override def foldable: Boolean = false
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override def toString: String = s"VAR_SAMP($child)"
    +}
    --- End diff --
    
    What will be the error message if we call this function when `spark.sql.useAggregate2=false`? It will be good to provide a meaning error message.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152209437
  
    Merged build finished. Test PASSed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149781462
  
    **[Test build #44039 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44039/consoleFull)** for PR 9003 at commit [`4ae234e`](https://github.com/apache/spark/commit/4ae234e3c8385e3149937821f5cdcee8fb85c053).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42668167
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    --- End diff --
    
    noticed you override getStatistics() in all sub-class, thought this adds complexity of reading this code, simply override eval() in specific sub-class (secondmoment, skewness, kurtosis) would be clear and starightforward. 


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433386
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    --- End diff --
    
    This is the same as after we rename properly and change `val` to `var`:
    
    ~~~
    mean += deltaN
    ~~~


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149721248
  
    @sethah The storage of `InternalRow` is an implicit contract. Exposing `InternalRow` to subclasses makes the code harder to understand. For example, to confirm the following code, one needs to go back to the implementation of `CentralMomentAgg`.
    
    ~~~scala
    val M0 = buffer.getDouble(mutableAggBufferOffset)
    val M2 = buffer.getDouble(mutableAggBufferOffset + 2)
    val M4 = buffer.getDouble(mutableAggBufferOffset + 4)
    ~~~
    
    I suggest adding an abstract method to `CentralMomemtAgg` called `Double getStatistic(n: Double, mean: Double, moments: Array[Double])` and implement `eval` in `CentralMomentAgg`:
    
    ~~~scala
    override final def eval(buffer: InternalRow): Any = {
      val n = buffer.getDouble(offset)
      val mean = buffer.getDouble(offset + 1)
      val moments = Array.ofDim(maxMomemt)
      moments[0] = 1.0
      moments[1] = 0.0
      // fill in other moments up to maxMoment
      getStatistic(n, mean, moments)
    }
    ~~~
    
    Then we can hide the storage of `InternalRow` from subclasses.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-146004396
  
    Can one of the admins verify this patch?


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152174382
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150636269
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149790453
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42790961
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala ---
    @@ -168,6 +191,28 @@ object QueryTest {
         return None
       }
     
    +  /**
    +   * Runs the plan and makes sure the answer is within absTol of the expected result.
    +   * @param actualAnswer the actual result in a [[Row]].
    +   * @param expectedAnswer the expected result in a[[Row]].
    +   * @param absTol the absolute tolerance between actual and expected answers.
    +   */
    +  protected def checkAggregatesWithTol(actualAnswer: Row, expectedAnswer: Row, absTol: Double) = {
    --- End diff --
    
    @holdenk Changed name to `checkAggregatesWithTol` and moved to query test. We can now use this for the various aggregate testing. I think ultimately a more elegant solution is needed, but this provides a way to add tolerancing for 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42813053
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    --- End diff --
    
    * Again, `n`, `mean`, and `centralMoments: Array[Double]` is better here. This is only called once, performance is not as important as readability and simplicity. See my previous comments on the API.
    * Need API doc for `moments`.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42393271
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,304 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  def cloneBufferAttributes: Seq[Attribute] = aggBufferAttributes.map(_.newInstance())
    --- End diff --
    
    cloneBufferAttributes is no longer 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152301262
  
    Thanks @mengxr , I will send a PR for Stddev. 


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150389761
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152286816
  
    **[Test build #44611 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44611/consoleFull)** for PR 9003 at commit [`f49ce5c`](https://github.com/apache/spark/commit/f49ce5c0d6594f3b5348eda75e9aefed10617066).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `case class Variance(child: Expression,`\n  * `case class VarianceSamp(child: Expression,`\n  * `case class VariancePop(child: Expression,`\n  * `case class Skewness(child: Expression,`\n  * `case class Kurtosis(child: Expression,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression1 `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression1 `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression1 `\n  * `case class VariancePop(child: Expression) extends UnaryExpression with AggregateExpression1 `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression1 `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42897850
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,330 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * Returns `Double.NaN` when N = 0 or N = 1
    + *  -third and fourth moments return `Double.NaN` when second moment is zero
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * Size of aggregation buffer.
    +   */
    +  private[this] val bufferSize = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(bufferSize) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  // buffer offsets
    +  private[this] val nOffset = mutableAggBufferOffset
    +  private[this] val meanOffset = mutableAggBufferOffset + 1
    +  private[this] val secondMomentOffset = mutableAggBufferOffset + 2
    +  private[this] val thirdMomentOffset = mutableAggBufferOffset + 3
    +  private[this] val fourthMomentOffset = mutableAggBufferOffset + 4
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +  private[this] var n = 0.0
    +  private[this] var mean = 0.0
    +  private[this] var m2 = 0.0
    +  private[this] var m3 = 0.0
    +  private[this] var m4 = 0.0
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until bufferSize) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +      }
    +      n = buffer.getDouble(nOffset)
    +      mean = buffer.getDouble(meanOffset)
    +
    +      n += 1.0
    +      buffer.setDouble(nOffset, n)
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(meanOffset, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(secondMomentOffset)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(secondMomentOffset, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(thirdMomentOffset)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(thirdMomentOffset, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(fourthMomentOffset)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(fourthMomentOffset, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(nOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(meanOffset)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +
    +    n = n1 + n2
    +    buffer1.setDouble(nOffset, n)
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    mean = mean1 + deltaN * n
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(secondMomentOffset)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      m2 = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(secondMomentOffset, m2)
    +    }
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(thirdMomentOffset)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      m3 = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(thirdMomentOffset, m3)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(fourthMomentOffset)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      m4 = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(fourthMomentOffset, m4)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   * @param centralMoments Length `momentOrder + 1` array of central moments needed to
    +   *                       compute the aggregate stat.
    +   */
    +  def getStatistic(n: Double, mean: Double, centralMoments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(nOffset)
    +    val mean = buffer.getDouble(meanOffset)
    +    val moments = Array.ofDim[Double](momentOrder + 1)
    +    moments(0) = n
    +    moments(1) = mean
    +    if (momentOrder >= 2) {
    +      moments(2) = buffer.getDouble(secondMomentOffset)
    +    }
    +    if (momentOrder >= 3) {
    +      moments(3) = buffer.getDouble(thirdMomentOffset)
    +    }
    +    if (momentOrder >= 4) {
    +      moments(4) = buffer.getDouble(fourthMomentOffset)
    +    }
    +
    +    getStatistic(n, mean, moments)
    +  }
    +}
    +
    +case class Variance(child: Expression,
    +    mutableAggBufferOffset: Int = 0,
    +    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance"
    +
    +  override protected val momentOrder = 2
    +
    +  override def getStatistic(n: Double, mean: Double, moments: Array[Double]): Double = {
    +    require(moments.length == momentOrder + 1,
    +      s"$prettyName requires ${momentOrder + 1} central moments, received: ${moments.length}")
    +
    +    if (n == 0.0 || n == 1.0) Double.NaN else moments(2) / (n - 1.0)
    +  }
    +}
    +
    +case class VarianceSamp(child: Expression,
    +    mutableAggBufferOffset: Int = 0,
    +    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_samp"
    +
    +  override protected val momentOrder = 2
    +
    +  override def getStatistic(n: Double, mean: Double, moments: Array[Double]): Double = {
    +    require(moments.length == momentOrder + 1,
    +      s"$prettyName requires ${momentOrder + 1} central moment, received: ${moments.length}")
    +
    +    if (n == 0.0 || n == 1.0) Double.NaN else moments(2) / (n - 1.0)
    +  }
    +}
    +
    +case class VariancePop(child: Expression,
    +    mutableAggBufferOffset: Int = 0,
    +    inputAggBufferOffset: Int = 0) extends CentralMomentAgg(child) {
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  override def prettyName: String = "variance_pop"
    +
    +  override protected val momentOrder = 2
    +
    +  override def getStatistic(n: Double, mean: Double, moments: Array[Double]): Double = {
    +    require(moments.length == momentOrder + 1,
    +      s"$prettyName requires ${momentOrder + 1} central moment, received: ${moments.length}")
    +
    +    if (n == 0.0 || n == 1.0) Double.NaN else moments(2) / n
    --- End diff --
    
    Remove `|| n == 1.0`. Population variance should be `0.0` when `n == 1.0`, which is the same as `moments(2) / 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152174392
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433400
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    --- End diff --
    
    We don't need the `else` branch because it is not used if `maxMoment >= 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149785650
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151212601
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150327278
  
    Build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152209441
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44595/
    Test PASSed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433376
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    --- End diff --
    
    * This is not exactly the first central moment, which is always zero. So better to call it `mean`.
    * Make it a `var` and same applies to `currentM2`, `currentM3`, and `currentM4`.
    * Since the updates are in place, so there is no need to call it `currentM2`. Just `m2` should be sufficient.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150713085
  
    **[Test build #44267 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44267/consoleFull)** for PR 9003 at commit [`cf8a14b`](https://github.com/apache/spark/commit/cf8a14bb24924079af2c30234a083a1e6c6d4c23).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150662563
  
    Merged build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152249331
  
    **[Test build #44611 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44611/consoleFull)** for PR 9003 at commit [`f49ce5c`](https://github.com/apache/spark/commit/f49ce5c0d6594f3b5348eda75e9aefed10617066).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433384
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    --- End diff --
    
    `delta`, `deltaN`, `delta2`, and `deltaN` are frequently used. We can store them as a `private[this] var`, which should simplify the logic a little bit.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149425504
  
    **[Test build #43961 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43961/consoleFull)** for PR 9003 at commit [`b180a28`](https://github.com/apache/spark/commit/b180a2857716ccf0e7c32d5aca0417f5a515d2cb).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42562659
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    --- End diff --
    
    Yes. This should simplify the implementation, and reduce the overhead of computing lower-order statistics to a few if checks.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149424316
  
    add to whitelist


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42584819
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    --- End diff --
    
    Using a `sufficientMoments` variable defined in subclasses we can make sure we don't get values from the buffer that are unused for a particular aggregate. This results in defining an Array of sufficient moments in the subclasses like
    
    ```scala
    override protected val sufficientMoments = Array(2, 4) // second and fourth moments needed
    ```
    
    Would appreciate feedback on this approach.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r41325055
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -88,6 +88,276 @@ case class Average(child: Expression) extends AlgebraicAggregate {
       }
     }
     
    +abstract class CentralMomentAgg(child: Expression) extends AlgebraicAggregate {
    +
    +  // specify the maximum order moment needed for the computation
    +  def maxMoment: Int
    +
    +  override def children: Seq[Expression] = child :: Nil
    +
    +  override def nullable: Boolean = true
    +
    +  // Return data type.
    +  override def dataType: DataType = resultType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  protected val resultType = DoubleType
    +
    +  protected val currentM4 = AttributeReference("currentM4", resultType)()
    +  protected val currentM3 = AttributeReference("currentM3", resultType)()
    +  protected val currentM2 = AttributeReference("currentM2", resultType)()
    +  protected val currentM1 = AttributeReference("currentM1", resultType)()
    +  protected val currentM0 = AttributeReference("currentM0", resultType)()
    +
    +  override val bufferAttributes = List(currentM4, currentM3, currentM2, currentM1, currentM0)
    +
    +  override val initialValues = Seq(
    +    /* currentM4 = */ Cast(Literal(0), resultType),
    +    /* currentM3 = */ Cast(Literal(0), resultType),
    +    /* currentM2 = */ Cast(Literal(0), resultType),
    +    /* currentM1 = */ Cast(Literal(0), resultType),
    +    /* currentM0 = */ Cast(Literal(0), resultType)
    +  )
    +
    +  override val updateExpressions = {
    +    lazy val updateM0: Expression = {
    +      Add(currentM0, Cast(Literal(1), resultType))
    +    }
    +    lazy val delta = Subtract(Cast(child, resultType), currentM1)
    +    lazy val deltaN = Divide(delta, updateM0)
    +
    +    lazy val updateM1: Expression = if (maxMoment >= 1) {
    +      Add(currentM1, Divide(delta, updateM0))
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM2: Expression = if (maxMoment >= 2) {
    +      Add(currentM2, Multiply(deltaN * delta, Subtract(updateM0, Cast(Literal(1), resultType))))
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM3: Expression = if (maxMoment >= 3) {
    +      currentM3 + deltaN * deltaN * delta * (updateM0 - Cast(Literal(1), resultType)) *
    +        (updateM0 - Cast(Literal(2), resultType)) -
    +        deltaN * currentM2 * Cast(Literal(3), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM4: Expression = if (maxMoment >= 4) {
    +      currentM4 + deltaN * deltaN * deltaN * delta * (updateM0 - Cast(Literal(1), resultType)) *
    +        (updateM0 * updateM0 - updateM0 * Cast(Literal(3), resultType) +
    +          Cast(Literal(3), resultType)) + deltaN * deltaN * currentM2 *
    +        Cast(Literal(6), resultType) - deltaN * currentM3 * Cast(Literal(4), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    Seq(
    +      /* currentM4 = */ If(IsNull(child), currentM4, updateM4),
    +      /* currentM3 = */ If(IsNull(child), currentM3, updateM3),
    +      /* currentM2 = */ If(IsNull(child), currentM2, updateM2),
    +      /* currentM1 = */ If(IsNull(child), currentM1, updateM1),
    +      /* currentM0 = */ If(IsNull(child), currentM0, updateM0)
    +    )
    +  }
    +
    +  override val mergeExpressions = {
    +
    +
    +    lazy val updateM0 = currentM0.left + currentM0.right
    +    lazy val delta = currentM1.right - currentM1.left
    +    lazy val deltaN = delta / updateM0
    +
    +    lazy val updateM1 = if (maxMoment >= 1) {
    +      currentM1.left + delta * (currentM0.right / updateM0)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM2 = if (maxMoment >= 2) {
    +      currentM2.left + currentM2.right + delta * deltaN * currentM0.left * currentM0.right
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM3 = if (maxMoment >= 3) {
    --- End diff --
    
    maxMoment variable is implemented with intent to avoid extra computation. For instance, if variance function calls `CentralMomentAgg` then the 3rd and 4th moments do not need to be computed. Would appreciate feedback/criticism of this implementation.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149365106
  
    @sethah Besides inline comments, we can also think about how to simplify the class inheritance. It would be better if we don't need to touch `InternalRow` in subclasses. Also, `Variance` is the same as `VarianceSamp`, shall we make one inherit another with a different `prettyName`?


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42812923
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    --- End diff --
    
    `numMoments` -> `bufferSize`


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150324367
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44164/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151212635
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42793409
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    --- End diff --
    
    Updated.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151248176
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44366/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152248065
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150721424
  
    **[Test build #44272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44272/consoleFull)** for PR 9003 at commit [`b86386a`](https://github.com/apache/spark/commit/b86386a78ab4b27ca9b6b84afe6db4973204e9e9).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149790455
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44046/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149785662
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150365530
  
    **[Test build #44165 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44165/consoleFull)** for PR 9003 at commit [`e0c5c88`](https://github.com/apache/spark/commit/e0c5c8864e2c75d612f2723020850894de8f9db5).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `abstract class SecondMoment(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Variance(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VarianceSamp(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class VariancePop(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Skewness(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression, mutableAggBufferOffset: Int = 0,`\n  * `case class Kurtosis(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Skewness(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class Variance(child: Expression) extends UnaryExpression with AggregateExpression `\n  * `case class VariancePop(child: Expression
 ) extends UnaryExpression with AggregateExpression `\n  * `case class VarianceSamp(child: Expression) extends UnaryExpression with AggregateExpression `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149788583
  
    @mengxr Addressed all comments and rearranged the class inheritances for case classes. I made another abstract class `SecondMoment` which all `Variance` classes inherit from - I think this will make the addition of the `StdDev` classes very simple.
    
    Items still to be addressed: 
    
    * I have not implemented using the `AggregateFunction1` interface since I believe it will be removed. Right now, I have added placeholders that just copy the code for `Average` so that the current code will compile. I'm not sure if I need to implement this or if we can just wait until the `AggregateFunction1` interface is removed.
    * I need to add tolerancing to the `checkAnswer` function in SQL test suite because aggregate stats do not result in exact answers for the test data (e.g. 0.0 != 5e-17). I will work on this soon.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42897843
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,330 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * Returns `Double.NaN` when N = 0 or N = 1
    + *  -third and fourth moments return `Double.NaN` when second moment is zero
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * Size of aggregation buffer.
    +   */
    +  private[this] val bufferSize = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(bufferSize) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  // buffer offsets
    +  private[this] val nOffset = mutableAggBufferOffset
    +  private[this] val meanOffset = mutableAggBufferOffset + 1
    +  private[this] val secondMomentOffset = mutableAggBufferOffset + 2
    +  private[this] val thirdMomentOffset = mutableAggBufferOffset + 3
    +  private[this] val fourthMomentOffset = mutableAggBufferOffset + 4
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +  private[this] var n = 0.0
    +  private[this] var mean = 0.0
    +  private[this] var m2 = 0.0
    +  private[this] var m3 = 0.0
    +  private[this] var m4 = 0.0
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until bufferSize) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +      }
    +      n = buffer.getDouble(nOffset)
    +      mean = buffer.getDouble(meanOffset)
    +
    +      n += 1.0
    +      buffer.setDouble(nOffset, n)
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(meanOffset, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(secondMomentOffset)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(secondMomentOffset, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(thirdMomentOffset)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(thirdMomentOffset, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(fourthMomentOffset)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(fourthMomentOffset, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(nOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(meanOffset)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +
    +    n = n1 + n2
    +    buffer1.setDouble(nOffset, n)
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    mean = mean1 + deltaN * n
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(secondMomentOffset)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      m2 = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(secondMomentOffset, m2)
    +    }
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(thirdMomentOffset)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      m3 = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(thirdMomentOffset, m3)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(fourthMomentOffset)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      m4 = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(fourthMomentOffset, m4)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   * @param centralMoments Length `momentOrder + 1` array of central moments needed to
    +   *                       compute the aggregate stat.
    +   */
    +  def getStatistic(n: Double, mean: Double, centralMoments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(nOffset)
    +    val mean = buffer.getDouble(meanOffset)
    +    val moments = Array.ofDim[Double](momentOrder + 1)
    +    moments(0) = n
    --- End diff --
    
    Thought `moments(0)` and `moments(1)` are just placeholders, we should use the correct values `moments(0) = 1.0` and `moments(1) = 0.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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42889558
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    --- End diff --
    
    Removed this def and instead pass all moments up to the maximum moment to the `getStatistic` 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150719854
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433414
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    --- End diff --
    
    `mean1`


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42584406
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    --- End diff --
    
    Done.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150327256
  
     Build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150662310
  
    **[Test build #44234 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44234/consoleFull)** for PR 9003 at commit [`fd3f4d6`](https://github.com/apache/spark/commit/fd3f4d6f9ba5124406a7078c9e7991bf91abdad6).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149260063
  
    @mengxr I am working on it, and I have incorporated changes from the note you posted on the [Jira](https://issues.apache.org/jira/browse/SPARK-10641) - thanks for that! I pushed some changes. I had already changed the existing implementation of stddev unfortunately, but I can revert those changes so that the PR is a bit smaller. I was thinking that since variance hasn't been merged yet, that I could include it in the PR, but if you think it's best to leave it out I can do that as well.
    
    I tested the imperative versions vs the codegen versions I had before and they are significantly faster. I will work on getting the changes for stddev reverted so that it will be ready for review. 


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150389765
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44179/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149425773
  
    **[Test build #43961 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43961/consoleFull)** for PR 9003 at commit [`b180a28`](https://github.com/apache/spark/commit/b180a2857716ccf0e7c32d5aca0417f5a515d2cb).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable `\n  * `case class Stddev(child: Expression) extends CentralMomentAgg(child) `\n  * `case class StddevSamp(child: Expression) extends CentralMomentAgg(child) `\n  * `case class StddevPop(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Variance(child: Expression) extends CentralMomentAgg(child) `\n  * `case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) `\n  * `case class VariancePop(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Skewness(child: Expression) extends CentralMomentAgg(child) `\n  * `case class Kurtosis(child: Expression) extends CentralMomentAgg(child) `\n  * `abstract class CentralMomentAgg1(child: Expression)`\n  * `case class Kurtosis(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class Skewness(child: Expression)
  extends CentralMomentAgg1(child) `\n  * `case class Variance(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class VariancePop(child: Expression) extends CentralMomentAgg1(child) `\n  * `case class VarianceSamp(child: Expression) extends CentralMomentAgg1(child) `\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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433416
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    --- End diff --
    
    It is useful to provide a link to the wikipedia page.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150365603
  
    Build finished. Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150322612
  
    Build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149783443
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44039/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150637284
  
    **[Test build #44234 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44234/consoleFull)** for PR 9003 at commit [`fd3f4d6`](https://github.com/apache/spark/commit/fd3f4d6f9ba5124406a7078c9e7991bf91abdad6).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151267064
  
    I guess this is still failing HiveComparisonTest due to small error in the variance. 
    
    [info]   !== HIVE - 1 row(s) ==                                            == CATALYST - 1 row(s) ==
    [info]   !500	130091	260.182	0	498	142.92680950752384	20428.072876000006   500	130091	260.182	0	498	142.92680950752384	20428.07287599999 (HiveComparisonTest.scala:433)
    
    abs(20428.072876000006 - 20428.07287599999) = 1.4552e-11
    
    Not sure if we need to change the test? 


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42677457
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -857,3 +857,329 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(mutableAggBufferOffset + 2, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 3, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(mutableAggBufferOffset + 4, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +    var secondMoment = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +    var thirdMoment = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +    var fourthMoment = 0.0
    +
    +    val n = n1 + n2
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    val mean = mean1 + deltaN * n2
    +
    +    buffer1.setDouble(mutableAggBufferOffset, n)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      secondMoment = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    }
    +
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      thirdMoment = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      fourthMoment = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   */
    +  def getStatistic(n: Double, moments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(mutableAggBufferOffset)
    +    val moments = sufficientMoments.map { momentIdx =>
    --- End diff --
    
    Per Xiangrui's comments about passing buffer to subclasses, I think it's necessary to do it this way. We don't have to pass the InternalRow object to the subclasses, and just pass the sufficient moments instead. See below discussion for more details.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433380
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    --- End diff --
    
    not needed if `maxMoment >= 3`


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433365
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    --- End diff --
    
    Missing API doc. It is useful to cite the Wikipedia page. My version can be find here: http://arxiv.org/abs/1510.04923.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42697800
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ---
    @@ -221,4 +221,40 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
           emptyTableData.agg(sumDistinct('a)),
           Row(null))
       }
    +
    +  test("moments") {
    +    checkAnswer(
    +      testData2.agg(skewness('a)),
    +      Row(0.0))
    +
    +    checkAnswer(
    --- End diff --
    
    So there are a few options for how to go about this. In spark-testing-base (which I'd like to eventually merge some of the functionality of) I made a modified version of checkAnswer which also takes tolerance and does tolerance based matching for floating point data (see https://github.com/holdenk/spark-testing-base/blob/master/src/main/scala/com/holdenkarau/spark/testing/DataFrameSuiteBase.scala & approxEqualDataFrames + approxEquals for comparing the rows). The other option would be, since for these tests its just a single value, just collect the result back, extract the value and do a regular assertion with tolerance. 
    
    I think collecting the result and just checking the single value is probably the best plan for this PR and eventually maybe looking at a more general version of check answer. 


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149787874
  
    **[Test build #44046 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44046/consoleFull)** for PR 9003 at commit [`62c515c`](https://github.com/apache/spark/commit/62c515cad1e2fb712256601f139d098fa4fb3749).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150365606
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44165/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42879337
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    --- End diff --
    
    I don't think we are going to support arbitrary-order moments. Kurtosis should be sufficient:)


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r41324423
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -88,6 +88,276 @@ case class Average(child: Expression) extends AlgebraicAggregate {
       }
     }
     
    +abstract class CentralMomentAgg(child: Expression) extends AlgebraicAggregate {
    +
    +  // specify the maximum order moment needed for the computation
    +  def maxMoment: Int
    +
    +  override def children: Seq[Expression] = child :: Nil
    +
    +  override def nullable: Boolean = true
    +
    +  // Return data type.
    +  override def dataType: DataType = resultType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  protected val resultType = DoubleType
    +
    +  protected val currentM4 = AttributeReference("currentM4", resultType)()
    +  protected val currentM3 = AttributeReference("currentM3", resultType)()
    +  protected val currentM2 = AttributeReference("currentM2", resultType)()
    +  protected val currentM1 = AttributeReference("currentM1", resultType)()
    +  protected val currentM0 = AttributeReference("currentM0", resultType)()
    +
    +  override val bufferAttributes = List(currentM4, currentM3, currentM2, currentM1, currentM0)
    +
    +  override val initialValues = Seq(
    +    /* currentM4 = */ Cast(Literal(0), resultType),
    +    /* currentM3 = */ Cast(Literal(0), resultType),
    +    /* currentM2 = */ Cast(Literal(0), resultType),
    +    /* currentM1 = */ Cast(Literal(0), resultType),
    +    /* currentM0 = */ Cast(Literal(0), resultType)
    +  )
    +
    +  override val updateExpressions = {
    +    lazy val updateM0: Expression = {
    +      Add(currentM0, Cast(Literal(1), resultType))
    +    }
    +    lazy val delta = Subtract(Cast(child, resultType), currentM1)
    +    lazy val deltaN = Divide(delta, updateM0)
    +
    +    lazy val updateM1: Expression = if (maxMoment >= 1) {
    +      Add(currentM1, Divide(delta, updateM0))
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM2: Expression = if (maxMoment >= 2) {
    +      Add(currentM2, Multiply(deltaN * delta, Subtract(updateM0, Cast(Literal(1), resultType))))
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM3: Expression = if (maxMoment >= 3) {
    +      currentM3 + deltaN * deltaN * delta * (updateM0 - Cast(Literal(1), resultType)) *
    +        (updateM0 - Cast(Literal(2), resultType)) -
    +        deltaN * currentM2 * Cast(Literal(3), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM4: Expression = if (maxMoment >= 4) {
    +      currentM4 + deltaN * deltaN * deltaN * delta * (updateM0 - Cast(Literal(1), resultType)) *
    +        (updateM0 * updateM0 - updateM0 * Cast(Literal(3), resultType) +
    +          Cast(Literal(3), resultType)) + deltaN * deltaN * currentM2 *
    +        Cast(Literal(6), resultType) - deltaN * currentM3 * Cast(Literal(4), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    Seq(
    +      /* currentM4 = */ If(IsNull(child), currentM4, updateM4),
    +      /* currentM3 = */ If(IsNull(child), currentM3, updateM3),
    +      /* currentM2 = */ If(IsNull(child), currentM2, updateM2),
    +      /* currentM1 = */ If(IsNull(child), currentM1, updateM1),
    +      /* currentM0 = */ If(IsNull(child), currentM0, updateM0)
    +    )
    +  }
    +
    +  override val mergeExpressions = {
    +
    +
    +    lazy val updateM0 = currentM0.left + currentM0.right
    +    lazy val delta = currentM1.right - currentM1.left
    +    lazy val deltaN = delta / updateM0
    +
    +    lazy val updateM1 = if (maxMoment >= 1) {
    +      currentM1.left + delta * (currentM0.right / updateM0)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM2 = if (maxMoment >= 2) {
    +      currentM2.left + currentM2.right + delta * deltaN * currentM0.left * currentM0.right
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM3 = if (maxMoment >= 3) {
    +      currentM3.left + currentM3.right + deltaN * deltaN * delta * currentM0.left *
    +        currentM0.right * (currentM0.left - currentM0.right) +
    +        deltaN * (currentM0.left * currentM2.right - currentM0.right * currentM2.left) *
    +          Cast(Literal(3), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM4 = if (maxMoment >= 4) {
    +      currentM4.left + currentM4.right + deltaN * deltaN * deltaN * delta * currentM0.left *
    +        currentM0.right * (currentM0.left * currentM0.left - currentM0.left * currentM0.right +
    +        currentM0.right * currentM0.right) + deltaN * deltaN * Cast(Literal(6), resultType) *
    +        (currentM0.left * currentM0.left * currentM2.right +
    +          currentM0.right * currentM0.right * currentM2.left) +
    +        deltaN * Cast(Literal(4), resultType) *
    +          (currentM0.left * currentM3.right - currentM0.right * currentM3.left)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    Seq(
    +      /* currentM4 = */ If(IsNull(currentM4.left), currentM4.right,
    +                          If(IsNull(currentM4.right), currentM4.left, updateM4)),
    +      /* currentM3 = */ If(IsNull(currentM3.left), currentM3.right,
    +                          If(IsNull(currentM3.right), currentM3.left, updateM3)),
    +      /* currentM2 = */ If(IsNull(currentM2.left), currentM2.right,
    +                          If(IsNull(currentM2.right), currentM2.left, updateM2)),
    +      /* currentM1 = */ If(IsNull(currentM1.left), currentM1.right,
    +                            If(IsNull(currentM1.right), currentM1.left, updateM1)),
    +      /* currentM0 = */ If(IsNull(currentM0.left), currentM0.right,
    +                            If(IsNull(currentM0.right), currentM0.left, updateM0))
    +    )
    +  }
    +}
    +
    +//case class Average(child: Expression) extends CentralMomentAgg(child) {
    --- End diff --
    
    Aggregate functions which depend on lower order moments can easily be implemented using the CentralMomentAgg base class. I have commented them out for now, but left them for discussion purposes.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150086706
  
    **[Test build #44119 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44119/consoleFull)** for PR 9003 at commit [`1594aba`](https://github.com/apache/spark/commit/1594abafec18dd0ba1c67c7a68c8ff017f88660a).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150323291
  
    **[Test build #44164 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44164/consoleFull)** for PR 9003 at commit [`2c9046f`](https://github.com/apache/spark/commit/2c9046ff952d80ff6c9382fc9e1ba31dc8ade7b8).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r41324846
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -734,10 +750,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
         )
       }
     
    +  test("skewness") {
    +    checkAnswer(
    +      sql("SELECT skewness(a) FROM testData2"),
    +      Row(0.0)
    +    )
    +  }
    +
    +  test("kurtosis") {
    --- End diff --
    
    Ditto about tolerancing.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433412
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    --- End diff --
    
    `n1`


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r41429601
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -88,6 +88,276 @@ case class Average(child: Expression) extends AlgebraicAggregate {
       }
     }
     
    +abstract class CentralMomentAgg(child: Expression) extends AlgebraicAggregate {
    +
    +  // specify the maximum order moment needed for the computation
    +  def maxMoment: Int
    +
    +  override def children: Seq[Expression] = child :: Nil
    +
    +  override def nullable: Boolean = true
    +
    +  // Return data type.
    +  override def dataType: DataType = resultType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  protected val resultType = DoubleType
    +
    +  protected val currentM4 = AttributeReference("currentM4", resultType)()
    +  protected val currentM3 = AttributeReference("currentM3", resultType)()
    +  protected val currentM2 = AttributeReference("currentM2", resultType)()
    +  protected val currentM1 = AttributeReference("currentM1", resultType)()
    +  protected val currentM0 = AttributeReference("currentM0", resultType)()
    +
    +  override val bufferAttributes = List(currentM4, currentM3, currentM2, currentM1, currentM0)
    +
    +  override val initialValues = Seq(
    +    /* currentM4 = */ Cast(Literal(0), resultType),
    +    /* currentM3 = */ Cast(Literal(0), resultType),
    +    /* currentM2 = */ Cast(Literal(0), resultType),
    +    /* currentM1 = */ Cast(Literal(0), resultType),
    +    /* currentM0 = */ Cast(Literal(0), resultType)
    +  )
    +
    +  override val updateExpressions = {
    +    lazy val updateM0: Expression = {
    +      Add(currentM0, Cast(Literal(1), resultType))
    +    }
    +    lazy val delta = Subtract(Cast(child, resultType), currentM1)
    +    lazy val deltaN = Divide(delta, updateM0)
    +
    +    lazy val updateM1: Expression = if (maxMoment >= 1) {
    +      Add(currentM1, Divide(delta, updateM0))
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM2: Expression = if (maxMoment >= 2) {
    +      Add(currentM2, Multiply(deltaN * delta, Subtract(updateM0, Cast(Literal(1), resultType))))
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM3: Expression = if (maxMoment >= 3) {
    +      currentM3 + deltaN * deltaN * delta * (updateM0 - Cast(Literal(1), resultType)) *
    +        (updateM0 - Cast(Literal(2), resultType)) -
    +        deltaN * currentM2 * Cast(Literal(3), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM4: Expression = if (maxMoment >= 4) {
    +      currentM4 + deltaN * deltaN * deltaN * delta * (updateM0 - Cast(Literal(1), resultType)) *
    +        (updateM0 * updateM0 - updateM0 * Cast(Literal(3), resultType) +
    +          Cast(Literal(3), resultType)) + deltaN * deltaN * currentM2 *
    +        Cast(Literal(6), resultType) - deltaN * currentM3 * Cast(Literal(4), resultType)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    Seq(
    +      /* currentM4 = */ If(IsNull(child), currentM4, updateM4),
    +      /* currentM3 = */ If(IsNull(child), currentM3, updateM3),
    +      /* currentM2 = */ If(IsNull(child), currentM2, updateM2),
    +      /* currentM1 = */ If(IsNull(child), currentM1, updateM1),
    +      /* currentM0 = */ If(IsNull(child), currentM0, updateM0)
    +    )
    +  }
    +
    +  override val mergeExpressions = {
    +
    +
    +    lazy val updateM0 = currentM0.left + currentM0.right
    +    lazy val delta = currentM1.right - currentM1.left
    +    lazy val deltaN = delta / updateM0
    +
    +    lazy val updateM1 = if (maxMoment >= 1) {
    +      currentM1.left + delta * (currentM0.right / updateM0)
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM2 = if (maxMoment >= 2) {
    +      currentM2.left + currentM2.right + delta * deltaN * currentM0.left * currentM0.right
    +    } else {
    +      Cast(Literal(0), resultType)
    +    }
    +
    +    lazy val updateM3 = if (maxMoment >= 3) {
    --- End diff --
    
    @sethah would define Enumeration to represent the moment 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42923519
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,332 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * Behavior:
    + *  - null values are ignored
    + *  - returns `Double.NaN` when the column contains `Double.NaN` values
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * Size of aggregation buffer.
    +   */
    +  private[this] val bufferSize = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(bufferSize) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  // buffer offsets
    +  private[this] val nOffset = mutableAggBufferOffset
    +  private[this] val meanOffset = mutableAggBufferOffset + 1
    +  private[this] val secondMomentOffset = mutableAggBufferOffset + 2
    +  private[this] val thirdMomentOffset = mutableAggBufferOffset + 3
    +  private[this] val fourthMomentOffset = mutableAggBufferOffset + 4
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +  private[this] var n = 0.0
    +  private[this] var mean = 0.0
    +  private[this] var m2 = 0.0
    +  private[this] var m3 = 0.0
    +  private[this] var m4 = 0.0
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until bufferSize) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +      }
    +
    +      n = buffer.getDouble(nOffset)
    +      mean = buffer.getDouble(meanOffset)
    +
    +      n += 1.0
    +      buffer.setDouble(nOffset, n)
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(meanOffset, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(secondMomentOffset)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(secondMomentOffset, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(thirdMomentOffset)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(thirdMomentOffset, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(fourthMomentOffset)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(fourthMomentOffset, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(nOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(meanOffset)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +
    +    n = n1 + n2
    +    buffer1.setDouble(nOffset, n)
    +    delta = mean2 - mean1
    +    deltaN = if (n == 0.0) 0.0 else delta / n
    --- End diff --
    
    Removed divide by zero case here, which was causing problems when number of partitions > number of samples.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149425776
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43961/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42454065
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    --- End diff --
    
    Just to be clear, if we make `m2`, `m3`, ... as vars, we'll need to initialize them outside the if statements like
    
    ```scala
    var m2 = 0.0
    var m3 = 0.0
    var m4 = 0.0
    ```
    
    Then we can do the `buffer.getDouble` and `buffer.setDouble` inside the if statement to avoid doing these when possible.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42820025
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    --- End diff --
    
    @mengxr If we do this, I'm wondering if it doesn't make sense to store constants for all moments offset. It would help get rid of ambiguous integer offsets and should be fine if we don't plan to extend this to arbitrary order moments.
    
    ```scala
    private[this] val nOffset = mutableAggBufferOffset
    ...
    private[this] val kurtosisOffset = mutableAggBufferOffset + 4
    ```


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150322595
  
     Build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42813001
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      var n = buffer.getDouble(mutableAggBufferOffset)
    +      var mean = buffer.getDouble(mutableAggBufferOffset + 1)
    +      var m2 = 0.0
    +      var m3 = 0.0
    +      var m4 = 0.0
    +
    +      n += 1.0
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(mutableAggBufferOffset, n)
    +      buffer.setDouble(mutableAggBufferOffset + 1, mean)
    --- End diff --
    
    If the `mutableAggBufferOffset` is set at the constructor, can we define `private[this]` constant for them, e.g., 'nOffset', 'meanOffset`? This saves some computations.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151211786
  
    @mengxr Corrected `variance` to yield the population variance. Tests should pass 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42923579
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,330 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * Returns `Double.NaN` when N = 0 or N = 1
    + *  -third and fourth moments return `Double.NaN` when second moment is zero
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * Size of aggregation buffer.
    +   */
    +  private[this] val bufferSize = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(bufferSize) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  // buffer offsets
    +  private[this] val nOffset = mutableAggBufferOffset
    +  private[this] val meanOffset = mutableAggBufferOffset + 1
    +  private[this] val secondMomentOffset = mutableAggBufferOffset + 2
    +  private[this] val thirdMomentOffset = mutableAggBufferOffset + 3
    +  private[this] val fourthMomentOffset = mutableAggBufferOffset + 4
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +  private[this] var n = 0.0
    +  private[this] var mean = 0.0
    +  private[this] var m2 = 0.0
    +  private[this] var m3 = 0.0
    +  private[this] var m4 = 0.0
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until bufferSize) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +      }
    +      n = buffer.getDouble(nOffset)
    +      mean = buffer.getDouble(meanOffset)
    +
    +      n += 1.0
    +      buffer.setDouble(nOffset, n)
    +      delta = updateValue - mean
    +      deltaN = delta / n
    +      mean += deltaN
    +      buffer.setDouble(meanOffset, mean)
    +
    +      if (momentOrder >= 2) {
    +        m2 = buffer.getDouble(secondMomentOffset)
    +        m2 += delta * (delta - deltaN)
    +        buffer.setDouble(secondMomentOffset, m2)
    +      }
    +
    +      if (momentOrder >= 3) {
    +        delta2 = delta * delta
    +        deltaN2 = deltaN * deltaN
    +        m3 = buffer.getDouble(thirdMomentOffset)
    +        m3 += -3.0 * deltaN * m2 + delta * (delta2 - deltaN2)
    +        buffer.setDouble(thirdMomentOffset, m3)
    +      }
    +
    +      if (momentOrder >= 4) {
    +        m4 = buffer.getDouble(fourthMomentOffset)
    +        m4 += -4.0 * deltaN * m3 - 6.0 * deltaN2 * m2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +        buffer.setDouble(fourthMomentOffset, m4)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge two central moment buffers.
    +   */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val n1 = buffer1.getDouble(nOffset)
    +    val n2 = buffer2.getDouble(inputAggBufferOffset)
    +    val mean1 = buffer1.getDouble(meanOffset)
    +    val mean2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +
    +    var secondMoment1 = 0.0
    +    var secondMoment2 = 0.0
    +
    +    var thirdMoment1 = 0.0
    +    var thirdMoment2 = 0.0
    +
    +    var fourthMoment1 = 0.0
    +    var fourthMoment2 = 0.0
    +
    +    n = n1 + n2
    +    buffer1.setDouble(nOffset, n)
    +    delta = mean2 - mean1
    +    deltaN = delta / n
    +    mean = mean1 + deltaN * n
    +    buffer1.setDouble(mutableAggBufferOffset + 1, mean)
    +
    +    // higher order moments computed according to:
    +    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics
    +    if (momentOrder >= 2) {
    +      secondMoment1 = buffer1.getDouble(secondMomentOffset)
    +      secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +      m2 = secondMoment1 + secondMoment2 + delta * deltaN * n1 * n2
    +      buffer1.setDouble(secondMomentOffset, m2)
    +    }
    +
    +    if (momentOrder >= 3) {
    +      thirdMoment1 = buffer1.getDouble(thirdMomentOffset)
    +      thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +      m3 = thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * n1 * n2 *
    +        (n1 - n2) + 3.0 * deltaN * (n1 * secondMoment2 - n2 * secondMoment1)
    +      buffer1.setDouble(thirdMomentOffset, m3)
    +    }
    +
    +    if (momentOrder >= 4) {
    +      fourthMoment1 = buffer1.getDouble(fourthMomentOffset)
    +      fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +      m4 = fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * n1 *
    +        n2 * (n1 * n1 - n1 * n2 + n2 * n2) + deltaN * deltaN * 6.0 *
    +        (n1 * n1 * secondMoment2 + n2 * n2 * secondMoment1) +
    +        4.0 * deltaN * (n1 * thirdMoment2 - n2 * thirdMoment1)
    +      buffer1.setDouble(fourthMomentOffset, m4)
    +    }
    +  }
    +
    +  /**
    +   * Compute aggregate statistic from sufficient moments.
    +   * @param centralMoments Length `momentOrder + 1` array of central moments needed to
    +   *                       compute the aggregate stat.
    +   */
    +  def getStatistic(n: Double, mean: Double, centralMoments: Array[Double]): Double
    +
    +  override final def eval(buffer: InternalRow): Any = {
    +    val n = buffer.getDouble(nOffset)
    +    val mean = buffer.getDouble(meanOffset)
    +    val moments = Array.ofDim[Double](momentOrder + 1)
    +    moments(0) = n
    --- End diff --
    
    Corrected.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150328874
  
    **[Test build #44165 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44165/consoleFull)** for PR 9003 at commit [`e0c5c88`](https://github.com/apache/spark/commit/e0c5c8864e2c75d612f2723020850894de8f9db5).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149254181
  
    @sethah, are you working on this? My suggestion is to implement skewness and kurtosis in a single aggregate function that implements ImperativeAggregate based on the discussion in SPARK-10953, but do not change existing implementation of standard deviation and variance. This makes the PR simpler and easier to review and merge.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42562765
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    +      secondMoment1 + secondMoment2 + delta * deltaN * zeroMoment1 * zeroMoment2
    +    } else {
    +      0.0
    +    }
    +
    +    val thirdMoment = if (maxMoment >= 3) {
    +      thirdMoment1 + thirdMoment2 + deltaN * deltaN * delta * zeroMoment1 * zeroMoment2 *
    +        (zeroMoment1 - zeroMoment2) + 3.0 * deltaN *
    +        (zeroMoment1 * secondMoment2 - zeroMoment2 * secondMoment1)
    +    } else {
    +      0.0
    +    }
    +
    +    val fourthMoment = if (maxMoment >= 4) {
    +      fourthMoment1 + fourthMoment2 + deltaN * deltaN * deltaN * delta * zeroMoment1 *
    +        zeroMoment2 * (zeroMoment1 * zeroMoment1 - zeroMoment1 * zeroMoment2 +
    +          zeroMoment2 * zeroMoment2) + deltaN * deltaN * 6.0 *
    +        (zeroMoment1 * zeroMoment1 * secondMoment2 + zeroMoment2 * zeroMoment2 * secondMoment1) +
    +        4.0 * deltaN * (zeroMoment1 * thirdMoment2 - zeroMoment2 * thirdMoment1)
    +    } else {
    +      0.0
    +    }
    +
    +    buffer1.setDouble(mutableAggBufferOffset, zeroMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 1, firstMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 2, secondMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 3, thirdMoment)
    +    buffer1.setDouble(mutableAggBufferOffset + 4, fourthMoment)
    +  }
    +}
    +
    +case class Stddev(child: Expression) extends CentralMomentAgg(child) {
    +
    +  override def prettyName: String = "stddev"
    +
    +  protected val maxMoment = 2
    +
    +  def eval(buffer: InternalRow): Any = {
    +    // stddev = sqrt(M2 / (M0 - 1))
    +    val M0 = buffer.getDouble(mutableAggBufferOffset)
    +    val M2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +
    +    if (M0 == 0.0) {
    +      0.0
    --- End diff --
    
    R is more correct on statistics, I 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433375
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    --- End diff --
    
    It is easier to call this `n` and make it a `var`.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-151213216
  
    **[Test build #44366 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44366/consoleFull)** for PR 9003 at commit [`3045e3b`](https://github.com/apache/spark/commit/3045e3b1d82ac73d154dc4d2165e920c74bdc118).


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42584447
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    --- End diff --
    
    done.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-149779451
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42897835
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    --- End diff --
    
    This is not for the case when `n = 0` or `n =1` but when we have `null` or `NaN` in the values. Are we ignoring them or outputting `NaN` directly?


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150719844
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42812933
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    --- End diff --
    
    Shall we move all variables before 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150105959
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44119/
    Test FAILed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150736271
  
    **[Test build #44272 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44272/consoleFull)** for PR 9003 at commit [`b86386a`](https://github.com/apache/spark/commit/b86386a78ab4b27ca9b6b84afe6db4973204e9e9).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42812895
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    --- End diff --
    
    Please also document the behavior for `null` and `NaN` 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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150712754
  
    Merged build started.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42433422
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -842,3 +699,302 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed
    +   */
    +  protected def maxMoment: Int
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  /**
    +   * Initialize all moments to zero
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    var aggIndex = 0
    +    while (aggIndex < numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +      aggIndex += 1
    +    }
    +  }
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    +      }
    +      val currentM0 = buffer.getDouble(mutableAggBufferOffset)
    +      val currentM1 = buffer.getDouble(mutableAggBufferOffset + 1)
    +      val currentM2 = buffer.getDouble(mutableAggBufferOffset + 2)
    +      val currentM3 = buffer.getDouble(mutableAggBufferOffset + 3)
    +      val currentM4 = buffer.getDouble(mutableAggBufferOffset + 4)
    +
    +      val updateM0 = currentM0 + 1.0
    +      val delta = updateValue - currentM1
    +      val deltaN = delta / updateM0
    +
    +      val updateM1 = currentM1 + delta / updateM0
    +      val updateM2 = if (maxMoment >= 2) {
    +        currentM2 + delta * (delta - deltaN)
    +      } else {
    +        0.0
    +      }
    +      val delta2 = delta * delta
    +      val deltaN2 = deltaN * deltaN
    +      val updateM3 = if (maxMoment >= 3) {
    +        currentM3 - 3.0 * deltaN * updateM2 + delta * (delta2 - deltaN2)
    +      } else {
    +        0.0
    +      }
    +      val updateM4 = if (maxMoment >= 4) {
    +        currentM4 - 4.0 * deltaN * updateM3 - 6.0 * deltaN2 * updateM2 +
    +          delta * (delta * delta2 - deltaN * deltaN2)
    +      } else {
    +        0.0
    +      }
    +
    +      buffer.setDouble(mutableAggBufferOffset, updateM0)
    +      buffer.setDouble(mutableAggBufferOffset + 1, updateM1)
    +      buffer.setDouble(mutableAggBufferOffset + 2, updateM2)
    +      buffer.setDouble(mutableAggBufferOffset + 3, updateM3)
    +      buffer.setDouble(mutableAggBufferOffset + 4, updateM4)
    +    }
    +  }
    +
    +  /** Merge two central moment buffers. */
    +  override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = {
    +    val zeroMoment1 = buffer1.getDouble(mutableAggBufferOffset)
    +    val zeroMoment2 = buffer2.getDouble(inputAggBufferOffset)
    +    val firstMoment1 = buffer1.getDouble(mutableAggBufferOffset + 1)
    +    val firstMoment2 = buffer2.getDouble(inputAggBufferOffset + 1)
    +    val secondMoment1 = buffer1.getDouble(mutableAggBufferOffset + 2)
    +    val secondMoment2 = buffer2.getDouble(inputAggBufferOffset + 2)
    +    val thirdMoment1 = buffer1.getDouble(mutableAggBufferOffset + 3)
    +    val thirdMoment2 = buffer2.getDouble(inputAggBufferOffset + 3)
    +    val fourthMoment1 = buffer1.getDouble(mutableAggBufferOffset + 4)
    +    val fourthMoment2 = buffer2.getDouble(inputAggBufferOffset + 4)
    +
    +    val zeroMoment = zeroMoment1 + zeroMoment2
    +    val delta = firstMoment2 - firstMoment1
    +    val deltaN = delta / zeroMoment
    +
    +    val firstMoment = firstMoment1 + deltaN * zeroMoment2
    +
    +    val secondMoment = if (maxMoment >= 2) {
    +      secondMoment1 + secondMoment2 + delta * deltaN * zeroMoment1 * zeroMoment2
    +    } else {
    --- End diff --
    
    not necessary to have the `else` branch


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][SQL] Add Skewness and Kurtosis S...

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

    https://github.com/apache/spark/pull/9003#issuecomment-152287004
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44611/
    Test PASSed.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#issuecomment-150712741
  
     Merged build triggered.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10641][WIP][SQL] Add Skewness and Kurto...

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

    https://github.com/apache/spark/pull/9003#discussion_r42812941
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala ---
    @@ -930,3 +930,327 @@ object HyperLogLogPlusPlus {
       )
       // scalastyle:on
     }
    +
    +/**
    + * A central moment is the expected value of a specified power of the deviation of a random
    + * variable from the mean. Central moments are often used to characterize the properties of about
    + * the shape of a distribution.
    + *
    + * This class implements online, one-pass algorithms for computing the central moments of a set of
    + * points.
    + *
    + * References:
    + *  - Xiangrui Meng.  "Simpler Online Updates for Arbitrary-Order Central Moments."
    + *      2015. http://arxiv.org/abs/1510.04923
    + *
    + * @see [[https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
    + *     Algorithms for calculating variance (Wikipedia)]]
    + *
    + * @param child to compute central moments of.
    + */
    +abstract class CentralMomentAgg(child: Expression) extends ImperativeAggregate with Serializable {
    +
    +  /**
    +   * The maximum central moment order to be computed.
    +   */
    +  protected def momentOrder: Int
    +
    +  /**
    +   * Array of sufficient moments need to compute the aggregate statistic.
    +   */
    +  protected def sufficientMoments: Array[Int]
    +
    +  override def children: Seq[Expression] = Seq(child)
    +
    +  override def nullable: Boolean = false
    +
    +  override def dataType: DataType = DoubleType
    +
    +  // Expected input data type.
    +  // TODO: Right now, we replace old aggregate functions (based on AggregateExpression1) to the
    +  // new version at planning time (after analysis phase). For now, NullType is added at here
    +  // to make it resolved when we have cases like `select avg(null)`.
    +  // We can use our analyzer to cast NullType to the default data type of the NumericType once
    +  // we remove the old aggregate functions. Then, we will not need NullType at here.
    +  override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))
    +
    +  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
    +
    +  /**
    +   * The number of central moments to store in the buffer.
    +   */
    +  private[this] val numMoments = 5
    +
    +  override val aggBufferAttributes: Seq[AttributeReference] = Seq.tabulate(numMoments) { i =>
    +    AttributeReference(s"M$i", DoubleType)()
    +  }
    +
    +  // Note: although this simply copies aggBufferAttributes, this common code can not be placed
    +  // in the superclass because that will lead to initialization ordering issues.
    +  override val inputAggBufferAttributes: Seq[AttributeReference] =
    +    aggBufferAttributes.map(_.newInstance())
    +
    +  /**
    +   * Initialize all moments to zero.
    +   */
    +  override def initialize(buffer: MutableRow): Unit = {
    +    for (aggIndex <- 0 until numMoments) {
    +      buffer.setDouble(mutableAggBufferOffset + aggIndex, 0.0)
    +    }
    +  }
    +
    +  // frequently used values for online updates
    +  private[this] var delta = 0.0
    +  private[this] var deltaN = 0.0
    +  private[this] var delta2 = 0.0
    +  private[this] var deltaN2 = 0.0
    +
    +  /**
    +   * Update the central moments buffer.
    +   */
    +  override def update(buffer: MutableRow, input: InternalRow): Unit = {
    +    val v = Cast(child, DoubleType).eval(input)
    +    if (v != null) {
    +      val updateValue = v match {
    +        case d: Double => d
    +        case _ => 0.0
    --- End diff --
    
    Could it happen? If this is not expected, we should throw an exception instead of treating it as `0.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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org