You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/12/02 02:38:38 UTC

[GitHub] [spark] infoankitp commented on a diff in pull request #38865: [SPARK-41232][SQL][PYTHON] Adding array_append function

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala:
##########
@@ -4600,3 +4600,69 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL
   override protected def withNewChildrenInternal(
     newLeft: Expression, newRight: Expression): ArrayExcept = copy(left = newLeft, right = newRight)
 }
+
+/**
+ * Given an array, and another element append the element at the end of the array.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(expr, expr) - Prepend the element",
+  examples = """
+    Examples:
+      > SELECT _FUNC_(array('b', 'd', 'c', 'a'), array(1, 2, 3, 4));
+
+  """,
+  since = "3.4.0",
+  group = "collection_funcs")
+case class ArrayAppend(left: Expression, right: Expression)
+    extends BinaryExpression
+    with ImplicitCastInputTypes {
+  override def prettyName: String = "array_append"
+  override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, AnyDataType)
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (left.dataType, right.dataType) match {
+      case (ArrayType(e1, _), (e2)) if e1.sameType(e2) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>

Review Comment:
   @LuciferYang Thanks for the review! Arrays are more strongly typed in Spark ! In case when we are getting mixed types of elements to append in the array, should we change the type of the array itself ?
   Also, we cannot create a column with Array[AnyDataType] like below, we will eventually get an error
   
   `scala> val df3 = Seq((Array("a", "b", 2, 5d), 3)).toDF("a", "b")
   org.apache.spark.SparkUnsupportedOperationException: No Encoder found for Any
   - array element class: "java.lang.Object"
   - field (class: "scala.Array", name: "_1")
   - root class: "scala.Tuple2"
   `
   
   Which is why I thought that its better to analyze the types of the array at the start itself and raise if the types do not match.



-- 
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