You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by hvanhovell <gi...@git.apache.org> on 2018/08/03 08:45:05 UTC

[GitHub] spark pull request #21965: [SPARK-23909][SQL] Add filter function.

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

    https://github.com/apache/spark/pull/21965#discussion_r207479758
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala ---
    @@ -210,3 +219,54 @@ case class ArrayTransform(
     
       override def prettyName: String = "transform"
     }
    +
    +/**
    + * Filters the input array using the given lambda function.
    + */
    +@ExpressionDescription(
    +  usage = "_FUNC_(expr, func) - Filters the input array using the given predicate.",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1);
    +       array(1, 3)
    +  """,
    +  since = "2.4.0")
    +case class ArrayFilter(
    +    input: Expression,
    +    function: Expression)
    +  extends ArrayBasedHigherOrderFunction with CodegenFallback {
    +
    +  override def nullable: Boolean = input.nullable
    +
    +  override def dataType: DataType = input.dataType
    +
    +  override def expectingFunctionType: AbstractDataType = BooleanType
    +
    +  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayFilter = {
    +    val elem = ArrayBasedHigherOrderFunction.elementArgumentType(input.dataType)
    +    copy(function = f(function, elem :: Nil))
    +  }
    +
    +  @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function
    +
    +  override def eval(input: InternalRow): Any = {
    +    val arr = this.input.eval(input).asInstanceOf[ArrayData]
    +    if (arr == null) {
    +      null
    +    } else {
    +      val f = functionForEval
    +      val buffer = new mutable.ArrayBuffer[Any]
    --- End diff --
    
    I am wondering if we should use the buffer builder with a size hint here? Or, alternatively manage the array ourself.


---

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