You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "infoankitp (via GitHub)" <gi...@apache.org> on 2023/03/29 13:47:16 UTC

[GitHub] [spark] infoankitp commented on a diff in pull request #40563: [SPARK-41232][SPARK-41233][FOLLOWUP] Refactor `array_append` and `array_prepend` with `RuntimeReplaceable`

infoankitp commented on code in PR #40563:
URL: https://github.com/apache/spark/pull/40563#discussion_r1151952607


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala:
##########
@@ -1400,120 +1400,24 @@ case class ArrayContains(left: Expression, right: Expression)
     copy(left = newLeft, right = newRight)
 }
 
-// scalastyle:off line.size.limit
-@ExpressionDescription(
-  usage = """
-      _FUNC_(array, element) - Add the element at the beginning of the array passed as first
-      argument. Type of element should be the same as the type of the elements of the array.
-      Null element is also prepended to the array. But if the array passed is NULL
-      output is NULL
-    """,
-  examples = """
-    Examples:
-      > SELECT _FUNC_(array('b', 'd', 'c', 'a'), 'd');
-       ["d","b","d","c","a"]
-      > SELECT _FUNC_(array(1, 2, 3, null), null);
-       [null,1,2,3,null]
-      > SELECT _FUNC_(CAST(null as Array<Int>), 2);
-       NULL
-  """,
-  group = "array_funcs",
-  since = "3.5.0")
-case class ArrayPrepend(left: Expression, right: Expression)
-  extends BinaryExpression
-    with ImplicitCastInputTypes
-    with ComplexTypeMergingExpression
-    with QueryErrorsBase {
-
-  override def nullable: Boolean = left.nullable
+trait InsertArrayOneSide extends RuntimeReplaceable
+  with ImplicitCastInputTypes with BinaryLike[Expression] with QueryErrorsBase {

Review Comment:
   Any diff between BinaryLike[Expression] and BinaryExpression? Also, any specific reason for removing ComplexTypeMergingExpression, I think this can help in assigning values of containsNull and nullable fields ?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala:
##########
@@ -5056,128 +4950,45 @@ case class ArrayCompact(child: Expression)
   """,
   since = "3.4.0",
   group = "array_funcs")
-case class ArrayAppend(left: Expression, right: Expression)
-  extends BinaryExpression
-    with ImplicitCastInputTypes
-    with ComplexTypeMergingExpression
-    with QueryErrorsBase {
-  override def prettyName: String = "array_append"
+case class ArrayAppend(left: Expression, right: Expression) extends InsertArrayOneSide {
 
-  @transient protected lazy val elementType: DataType =
-    inputTypes.head.asInstanceOf[ArrayType].elementType
+  override lazy val replacement: Expression =
+    ArrayInsert(left, Add(ArraySize(left), Literal(1)), right)

Review Comment:
   Inspite of ArraySize(left), I think ArrayInsert supports negative values, how about using Literal(-1), to select the position.



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala:
##########
@@ -1855,50 +1855,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     checkEvaluation(ArrayRepeat(Literal("hi"), Literal(null, IntegerType)), null)
   }
 
-  test("SPARK-41233: ArrayPrepend") {

Review Comment:
   I hope we have reviewed that we are not missing any type of test cases in DataFrameFunctionsSuite from these. If we are missing, please help add those. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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