Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14136#discussion_r89672415
  
    --- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala
 ---
    @@ -0,0 +1,326 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.expressions.aggregate
    +
    +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, 
DataInputStream, DataOutputStream}
    +import java.util
    +
    +import org.apache.spark.sql.AnalysisException
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
    +import 
org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, 
TypeCheckSuccess}
    +import org.apache.spark.sql.catalyst.expressions._
    +import 
org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.Countings
    +import org.apache.spark.sql.catalyst.util._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.collection.OpenHashMap
    +
    +
    +/**
    + * The Percentile aggregate function returns the exact percentile(s) of 
numeric column `expr` at
    + * the given percentage(s) with value range in [0.0, 1.0].
    + *
    + * The operator is bound to the slower sort based aggregation path because 
the number of elements
    + * and their partial order cannot be determined in advance. Therefore we 
have to store all the
    + * elements in memory, and that too many elements can cause GC paused and 
eventually OutOfMemory
    + * Errors.
    + *
    + * @param child child expression that produce numeric column value with 
`child.eval(inputRow)`
    + * @param percentageExpression Expression that represents a single 
percentage value or an array of
    + *                             percentage values. Each percentage value 
must be in the range
    + *                             [0.0, 1.0].
    + */
    +@ExpressionDescription(
    +  usage =
    +    """
    +      _FUNC_(col, percentage) - Returns the exact percentile value of 
numeric column `col` at the
    +      given percentage. The value of percentage must be between 0.0 and 
1.0.
    +
    +      _FUNC_(col, array(percentage1 [, percentage2]...)) - Returns the 
exact percentile value array
    +      of numeric column `col` at the given percentage(s). Each value of 
the percentage array must
    +      be between 0.0 and 1.0.
    +    """)
    +case class Percentile(
    +  child: Expression,
    +  percentageExpression: Expression,
    +  mutableAggBufferOffset: Int = 0,
    +  inputAggBufferOffset: Int = 0) extends 
TypedImperativeAggregate[Countings] {
    +
    +  def this(child: Expression, percentageExpression: Expression) = {
    +    this(child, percentageExpression, 0, 0)
    +  }
    +
    +  override def prettyName: String = "percentile"
    +
    +  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: 
Int): Percentile =
    +    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
    +
    +  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): 
Percentile =
    +    copy(inputAggBufferOffset = newInputAggBufferOffset)
    +
    +  // Mark as lazy so that percentageExpression is not evaluated during 
tree transformation.
    +  private lazy val returnPercentileArray = 
percentageExpression.dataType.isInstanceOf[ArrayType]
    +
    +  @transient
    +  private lazy val percentages = evalPercentages(percentageExpression)
    +
    +  override def children: Seq[Expression] = child :: percentageExpression 
:: Nil
    +
    +  // Returns null for empty inputs
    +  override def nullable: Boolean = true
    +
    +  override lazy val dataType: DataType = percentageExpression.dataType 
match {
    +    case _: ArrayType => ArrayType(DoubleType, false)
    +    case _ => DoubleType
    +  }
    +
    +  override def inputTypes: Seq[AbstractDataType] = 
percentageExpression.dataType match {
    +    case _: ArrayType => Seq(NumericType, ArrayType(DoubleType, false))
    +    case _ => Seq(NumericType, DoubleType)
    +  }
    +
    +  // Check the inputTypes are valid, and the percentageExpression 
satisfies:
    +  // 1. percentageExpression must be foldable;
    +  // 2. percentages(s) must be in the range [0.0, 1.0].
    +  override def checkInputDataTypes(): TypeCheckResult = {
    +    // Validate the inputTypes
    +    val defaultCheck = super.checkInputDataTypes()
    +    if (defaultCheck.isFailure) {
    +      defaultCheck
    +    } else if (!percentageExpression.foldable) {
    +      // percentageExpression must be foldable
    +      TypeCheckFailure(s"The percentage(s) must be a constant literal, " +
    +        s"but got ${percentageExpression}")
    +    } else if (percentages.exists(percentage => percentage < 0.0 || 
percentage > 1.0)) {
    +      // percentages(s) must be in the range [0.0, 1.0]
    +      TypeCheckFailure(s"Percentage(s) must be between 0.0 and 1.0, " +
    +        s"but got ${percentageExpression}")
    +    } else {
    +      TypeCheckSuccess
    +    }
    +  }
    +
    +  override def createAggregationBuffer(): Countings = {
    +    // Initialize new Countings instance here.
    +    Countings()
    +  }
    +
    +  private def evalPercentages(expr: Expression): Seq[Double] = 
(expr.dataType, expr.eval()) match {
    --- End diff --
    
    Move this to the definition of percentages. You can also make this much 
simpler. The analyzer guarantees that you either get a single double, or an 
ArrayData of double:
    ```scala
    @transient
    private lazy val percentages = percentageExpression.eval() match {
      case p: Double => Seq(p)
      case a: ArrayData => a.toDoubleArray().toSeq
    }
    ```


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

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

Reply via email to