-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23917][SQL] Add array_max function #21024
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
93c5b8a
a296bc0
c8c1d03
d017ccf
e082f00
a4fd616
a78c752
8dfd263
e739a0a
1cde795
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,7 @@ import java.util.Comparator | |
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.analysis.TypeCheckResult | ||
| import org.apache.spark.sql.catalyst.expressions.codegen._ | ||
| import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, MapData} | ||
| import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, MapData, TypeUtils} | ||
| import org.apache.spark.sql.types._ | ||
|
|
||
| /** | ||
|
|
@@ -287,3 +287,61 @@ case class ArrayContains(left: Expression, right: Expression) | |
|
|
||
| override def prettyName: String = "array_contains" | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * Returns the maximum value in the array. | ||
| */ | ||
| @ExpressionDescription( | ||
| usage = "_FUNC_(array) - Returns the maximum value in the array.", | ||
| examples = """ | ||
| Examples: | ||
| > SELECT _FUNC_(array(1, 20, null, 3)); | ||
| 20 | ||
| """, since = "2.4.0") | ||
| case class ArrayMax(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { | ||
|
|
||
| override def nullable: Boolean = | ||
| child.nullable || child.dataType.asInstanceOf[ArrayType].containsNull | ||
|
||
|
|
||
| override def foldable: Boolean = child.foldable | ||
|
||
|
|
||
| override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType) | ||
|
|
||
| private lazy val ordering = TypeUtils.getInterpretedOrdering(dataType) | ||
|
|
||
| override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { | ||
| val childGen = child.genCode(ctx) | ||
| val javaType = CodeGenerator.javaType(dataType) | ||
| val i = ctx.freshName("i") | ||
| val item = ExprCode("", | ||
| isNull = StatementValue(s"${childGen.value}.isNullAt($i)", "boolean"), | ||
| value = StatementValue(CodeGenerator.getValue(childGen.value, dataType, i), javaType)) | ||
| ev.copy(code = | ||
| s""" | ||
| |${childGen.code} | ||
| |boolean ${ev.isNull} = true; | ||
| |$javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need to use
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nvm, |
||
| |if (!${childGen.isNull}) { | ||
| | for (int $i = 0; $i < ${childGen.value}.numElements(); $i ++) { | ||
| | ${ctx.reassignIfGreater(dataType, ev, item)} | ||
| | } | ||
| |} | ||
| """.stripMargin) | ||
| } | ||
|
|
||
| override protected def nullSafeEval(input: Any): Any = { | ||
| var max: Any = null | ||
| input.asInstanceOf[ArrayData].foreach(dataType, (_, item) => | ||
| if (item != null && (max == null || ordering.gt(item, max))) { | ||
| max = item | ||
| } | ||
| ) | ||
| max | ||
| } | ||
|
|
||
| override def dataType: DataType = child.dataType match { | ||
| case ArrayType(dt, _) => dt | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should also check if
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added the check in the |
||
| case _ => throw new IllegalStateException("array_max accepts only arrays.") | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
quick nit
,(->, (